From dee530514ce0d50f6369bb087a480066b028b78a Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Wed, 24 Dec 2014 08:50:31 -0800 Subject: [PATCH] Revert "Merge hierarchy store into Tachyon [TACHYON-33 sub-task 5 & 6]" --- conf/tachyon-env.sh.template | 6 +- core/src/main/java/tachyon/Format.java | 19 +- core/src/main/java/tachyon/StorageDirId.java | 13 - core/src/main/java/tachyon/Users.java | 1 - .../{worker => client}/BlockHandler.java | 18 +- .../{worker => client}/BlockHandlerLocal.java | 13 +- .../java/tachyon/client/BlockInStream.java | 3 - .../java/tachyon/client/BlockOutStream.java | 71 +- .../main/java/tachyon/client/ReadType.java | 16 +- .../tachyon/client/RemoteBlockInStream.java | 16 +- .../main/java/tachyon/client/TachyonFS.java | 286 +- .../main/java/tachyon/client/TachyonFile.java | 43 +- .../main/java/tachyon/conf/WorkerConf.java | 17 +- .../main/java/tachyon/master/BlockInfo.java | 22 +- .../main/java/tachyon/master/InodeFile.java | 12 +- .../java/tachyon/master/MasterClient.java | 19 +- .../main/java/tachyon/master/MasterInfo.java | 71 +- .../tachyon/master/MasterServiceHandler.java | 15 +- .../java/tachyon/thrift/ClientBlockInfo.java | 195 +- .../tachyon/thrift/ClientDependencyInfo.java | 96 +- .../java/tachyon/thrift/ClientFileInfo.java | 32 +- .../tachyon/thrift/ClientLocationInfo.java | 486 -- .../src/main/java/tachyon/thrift/Command.java | 32 +- .../java/tachyon/thrift/MasterService.java | 929 ++-- ...va => OutOfMemoryForPinFileException.java} | 56 +- .../java/tachyon/thrift/WorkerService.java | 4112 +++-------------- .../main/java/tachyon/util/CommonUtils.java | 38 - .../java/tachyon/worker/BlocksLocker.java | 56 +- .../java/tachyon/worker/TachyonWorker.java | 26 +- .../java/tachyon/worker/WorkerClient.java | 129 +- .../tachyon/worker/WorkerServiceHandler.java | 76 +- .../java/tachyon/worker/WorkerStorage.java | 613 ++- .../tachyon/worker/hierarchy/StorageDir.java | 157 +- .../tachyon/worker/hierarchy/StorageTier.java | 33 +- .../tachyon/worker/netty/BlockResponse.java | 33 +- .../worker/netty/DataServerHandler.java | 45 +- .../tachyon/worker/netty/NettyDataServer.java | 6 +- .../tachyon/worker/netty/PipelineHandler.java | 7 +- .../tachyon/worker/nio/DataServerMessage.java | 64 +- .../tachyon/worker/nio/NIODataServer.java | 38 +- .../BlockHandlerLocalTest.java | 20 +- .../java/tachyon/client/TachyonFSTest.java | 9 +- .../java/tachyon/master/BlockInfoTest.java | 22 +- .../java/tachyon/master/InodeFileTest.java | 23 +- .../tachyon/master/LocalTachyonCluster.java | 38 +- .../LocalTachyonClusterMultiMaster.java | 32 +- .../java/tachyon/worker/DataServerTest.java | 5 +- .../worker/WorkerServiceHandlerTest.java | 93 +- .../tachyon/worker/WorkerStorageTest.java | 16 +- .../hierarchy/AllocateStrategyTest.java | 44 +- .../worker/hierarchy/EvictStrategyTest.java | 16 +- .../worker/hierarchy/HierarchyStoreTest.java | 115 - .../worker/hierarchy/StorageDirTest.java | 46 +- .../worker/hierarchy/StorageTierTest.java | 35 +- core/src/thrift/tachyon.thrift | 38 +- 55 files changed, 1773 insertions(+), 6699 deletions(-) rename core/src/main/java/tachyon/{worker => client}/BlockHandler.java (88%) rename core/src/main/java/tachyon/{worker => client}/BlockHandlerLocal.java (93%) delete mode 100644 core/src/main/java/tachyon/thrift/ClientLocationInfo.java rename core/src/main/java/tachyon/thrift/{OutOfSpaceException.java => OutOfMemoryForPinFileException.java} (81%) rename core/src/test/java/tachyon/{worker => client}/BlockHandlerLocalTest.java (85%) delete mode 100644 core/src/test/java/tachyon/worker/hierarchy/HierarchyStoreTest.java diff --git a/conf/tachyon-env.sh.template b/conf/tachyon-env.sh.template index ee49485e1008..fa7eec125a26 100755 --- a/conf/tachyon-env.sh.template +++ b/conf/tachyon-env.sh.template @@ -58,16 +58,12 @@ CONF_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" export TACHYON_JAVA_OPTS+=" -Dlog4j.configuration=file:$CONF_DIR/log4j.properties -Dtachyon.debug=false - -Dtachyon.worker.hierarchystore.level.max=1 - -Dtachyon.worker.hierarchystore.level0.alias=MEM - -Dtachyon.worker.hierarchystore.level0.dirs.path=$TACHYON_RAM_FOLDER - -Dtachyon.worker.hierarchystore.level0.dirs.quota=$TACHYON_WORKER_MEMORY_SIZE -Dtachyon.underfs.address=$TACHYON_UNDERFS_ADDRESS -Dtachyon.underfs.hdfs.impl=$TACHYON_UNDERFS_HDFS_IMPL -Dtachyon.data.folder=$TACHYON_UNDERFS_ADDRESS/tmp/tachyon/data -Dtachyon.workers.folder=$TACHYON_UNDERFS_ADDRESS/tmp/tachyon/workers -Dtachyon.worker.memory.size=$TACHYON_WORKER_MEMORY_SIZE - -Dtachyon.worker.data.folder=/tachyonworker/ + -Dtachyon.worker.data.folder=$TACHYON_RAM_FOLDER/tachyonworker/ -Dtachyon.master.worker.timeout.ms=60000 -Dtachyon.master.hostname=$TACHYON_MASTER_ADDRESS -Dtachyon.master.journal.folder=$TACHYON_HOME/journal/ diff --git a/core/src/main/java/tachyon/Format.java b/core/src/main/java/tachyon/Format.java index 0ba60c5ddd28..30576aa68bdc 100644 --- a/core/src/main/java/tachyon/Format.java +++ b/core/src/main/java/tachyon/Format.java @@ -66,18 +66,13 @@ public static void main(String[] args) throws IOException { + System.currentTimeMillis()); } else if (args[0].toUpperCase().equals("WORKER")) { WorkerConf workerConf = WorkerConf.get(); - for (int level = 0; level < workerConf.MAX_HIERARCHY_STORAGE_LEVEL; level ++) { - String[] dirPaths = workerConf.STORAGE_TIER_DIRS[level].split(","); - for (int i = 0; i < dirPaths.length; i ++) { - String dataPath = CommonUtils.concat(dirPaths[i].trim(), workerConf.DATA_FOLDER); - UnderFileSystem ufs = UnderFileSystem.get(dataPath); - System.out.println("Removing data under folder: " + dataPath); - if (ufs.exists(dataPath)) { - String[] files = ufs.list(dataPath); - for (String file : files) { - ufs.delete(CommonUtils.concat(dataPath, file), true); - } - } + String localFolder = workerConf.DATA_FOLDER; + UnderFileSystem ufs = UnderFileSystem.get(localFolder); + System.out.println("Removing local data under folder: " + localFolder); + if (ufs.exists(localFolder)) { + String[] files = ufs.list(localFolder); + for (String file : files) { + ufs.delete(CommonUtils.concat(localFolder, file), true); } } } else { diff --git a/core/src/main/java/tachyon/StorageDirId.java b/core/src/main/java/tachyon/StorageDirId.java index 009022e15858..e0ad75f08f5c 100644 --- a/core/src/main/java/tachyon/StorageDirId.java +++ b/core/src/main/java/tachyon/StorageDirId.java @@ -21,19 +21,6 @@ public class StorageDirId { static final long UNKNOWN = -1; - /** - * Compare storage level of StorageDirs - * - * @param storageDirIdLeft The left value of StorageDirId to be compared - * @param storageDirIdRight The right value of StorageDirId to be compared - * @return negative if storage level of left StorageDirId is higher than the right one, zero if - * equals, positive if lower. - */ - public static int compareStorageLevel(long storageDirIdLeft, long storageDirIdRight) { - return getStorageLevelAliasValue(storageDirIdLeft) - - getStorageLevelAliasValue(storageDirIdRight); - } - /** * Generate StorageDirId from given information * diff --git a/core/src/main/java/tachyon/Users.java b/core/src/main/java/tachyon/Users.java index b72f61c7eed3..b6f025eb2108 100644 --- a/core/src/main/java/tachyon/Users.java +++ b/core/src/main/java/tachyon/Users.java @@ -38,7 +38,6 @@ public class Users { public static final int DATASERVER_USER_ID = -1; public static final int CHECKPOINT_USER_ID = -2; - public static final int EVICT_USER_ID = -3; private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); diff --git a/core/src/main/java/tachyon/worker/BlockHandler.java b/core/src/main/java/tachyon/client/BlockHandler.java similarity index 88% rename from core/src/main/java/tachyon/worker/BlockHandler.java rename to core/src/main/java/tachyon/client/BlockHandler.java index 8f76766ff689..a00b3186ce21 100644 --- a/core/src/main/java/tachyon/worker/BlockHandler.java +++ b/core/src/main/java/tachyon/client/BlockHandler.java @@ -13,12 +13,11 @@ * the License. */ -package tachyon.worker; +package tachyon.client; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.ByteChannel; import tachyon.TachyonURI; @@ -76,21 +75,6 @@ public int append(long blockOffset, byte[] buf, int offset, int length) throws I */ public abstract boolean delete() throws IOException; - /** - * Get channel used to access block file - * - * @return the channel bounded with the block file - */ - public abstract ByteChannel getChannel(); - - /** - * Get length of the block file - * - * @return the length of the block file - * @throws IOException - */ - public abstract long getLength() throws IOException; - /** * Read data from block file * diff --git a/core/src/main/java/tachyon/worker/BlockHandlerLocal.java b/core/src/main/java/tachyon/client/BlockHandlerLocal.java similarity index 93% rename from core/src/main/java/tachyon/worker/BlockHandlerLocal.java rename to core/src/main/java/tachyon/client/BlockHandlerLocal.java index 8fdaf230c3b9..86cf8f39e5e3 100644 --- a/core/src/main/java/tachyon/worker/BlockHandlerLocal.java +++ b/core/src/main/java/tachyon/client/BlockHandlerLocal.java @@ -13,13 +13,12 @@ * the License. */ -package tachyon.worker; +package tachyon.client; import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.ByteBuffer; -import java.nio.channels.ByteChannel; import java.nio.channels.FileChannel; import java.nio.channels.FileChannel.MapMode; @@ -84,16 +83,6 @@ public boolean delete() throws IOException { return new File(mFilePath).delete(); } - @Override - public ByteChannel getChannel() { - return mLocalFileChannel; - } - - @Override - public long getLength() throws IOException { - return mLocalFile.length(); - } - @Override public ByteBuffer read(long blockOffset, int length) throws IOException { long fileLength = mLocalFile.length(); diff --git a/core/src/main/java/tachyon/client/BlockInStream.java b/core/src/main/java/tachyon/client/BlockInStream.java index 662d63045288..3b14b211d4eb 100644 --- a/core/src/main/java/tachyon/client/BlockInStream.java +++ b/core/src/main/java/tachyon/client/BlockInStream.java @@ -53,9 +53,6 @@ public static BlockInStream get(TachyonFile tachyonFile, ReadType readType, int Object ufsConf) throws IOException { TachyonByteBuffer buf = tachyonFile.readLocalByteBuffer(blockIndex); if (buf != null) { - if (readType.isPromote()) { - tachyonFile.promoteBlock(blockIndex); - } return new LocalBlockInStream(tachyonFile, readType, blockIndex, buf); } diff --git a/core/src/main/java/tachyon/client/BlockOutStream.java b/core/src/main/java/tachyon/client/BlockOutStream.java index 17ee44f56cf8..4cfcca3e9274 100644 --- a/core/src/main/java/tachyon/client/BlockOutStream.java +++ b/core/src/main/java/tachyon/client/BlockOutStream.java @@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory; import tachyon.Constants; -import tachyon.thrift.ClientLocationInfo; import tachyon.util.CommonUtils; /** @@ -44,7 +43,6 @@ public class BlockOutStream extends OutStream { private long mInFileBytes = 0; private long mWrittenBytes = 0; - private ClientLocationInfo mLocationInfo = null; private String mLocalFilePath = null; private RandomAccessFile mLocalFile = null; @@ -83,19 +81,28 @@ public class BlockOutStream extends OutStream { throw new IOException(msg); } + File localFolder = mTachyonFS.createAndGetUserLocalTempFolder(); + if (localFolder == null) { + mCanWrite = false; + String msg = "Failed to create temp user folder for tachyon client."; + throw new IOException(msg); + } + + mLocalFilePath = CommonUtils.concat(localFolder.getPath(), mBlockId); + mLocalFile = new RandomAccessFile(mLocalFilePath, "rw"); + mLocalFileChannel = mLocalFile.getChannel(); + // change the permission of the temporary file in order that the worker can move it. + CommonUtils.changeLocalFileToFullPermission(mLocalFilePath); + // use the sticky bit, only the client and the worker can write to the block + CommonUtils.setLocalFileStickyBit(mLocalFilePath); + LOG.info(mLocalFilePath + " was created!"); + mBuffer = ByteBuffer.allocate(mUserConf.FILE_BUFFER_BYTES + 4); } private synchronized void appendCurrentBuffer(byte[] buf, int offset, int length) throws IOException { - boolean reqResult = false; - if (mLocationInfo == null) { - mLocationInfo = mTachyonFS.requestSpace(length); - reqResult = (mLocationInfo != null); - } else { - reqResult = mTachyonFS.requestSpace(mLocationInfo.getStorageDirId(), length); - } - if (!reqResult) { + if (!mTachyonFS.requestSpace(length)) { mCanWrite = false; String msg = @@ -104,17 +111,6 @@ private synchronized void appendCurrentBuffer(byte[] buf, int offset, int length throw new IOException(msg); } - if (mLocalFilePath == null) { - String localTempFolder = createUserLocalTempFolder(); - mLocalFilePath = CommonUtils.concat(localTempFolder, mBlockId); - mLocalFile = new RandomAccessFile(mLocalFilePath, "rw"); - mLocalFileChannel = mLocalFile.getChannel(); - // change the permission of the temporary file in order that the worker can move it. - CommonUtils.changeLocalFileToFullPermission(mLocalFilePath); - // use the sticky bit, only the client and the worker can write to the block - CommonUtils.setLocalFileStickyBit(mLocalFilePath); - LOG.info(mLocalFilePath + " was created!"); - } MappedByteBuffer out = mLocalFileChannel.map(MapMode.READ_WRITE, mInFileBytes, length); out.put(buf, offset, length); @@ -147,39 +143,16 @@ public void close() throws IOException { } if (mCancel) { - if (mLocationInfo != null) { // if file was written - mTachyonFS.releaseSpace(mLocationInfo.getStorageDirId(), - mWrittenBytes - mBuffer.position()); - new File(mLocalFilePath).delete(); - LOG.info("Canceled output of block " + mBlockId + ", deleted local file " - + mLocalFilePath); - } - } else if (mLocationInfo != null) { - mTachyonFS.cacheBlock(mLocationInfo.getStorageDirId(), mBlockId); + mTachyonFS.releaseSpace(mWrittenBytes - mBuffer.position()); + new File(mLocalFilePath).delete(); + LOG.info("Canceled output of block " + mBlockId + ", deleted local file " + mLocalFilePath); + } else { + mTachyonFS.cacheBlock(mBlockId); } } mClosed = true; } - /** - * Create temporary folder for the user in some StorageDir - * - * @return the path of the folder - * @throws IOException - */ - private String createUserLocalTempFolder() throws IOException { - String localTempFolder = null; - if (mLocationInfo != null) { - localTempFolder = mTachyonFS.createAndGetUserLocalTempFolder(mLocationInfo.getPath()); - } - - if (localTempFolder == null) { - mCanWrite = false; - throw new IOException("Failed to create temp user folder for tachyon client."); - } - return localTempFolder; - } - @Override public void flush() throws IOException { // Since this only writes to memory, this flush is not outside visible. diff --git a/core/src/main/java/tachyon/client/ReadType.java b/core/src/main/java/tachyon/client/ReadType.java index fe9ec6b2046e..a088092c746f 100644 --- a/core/src/main/java/tachyon/client/ReadType.java +++ b/core/src/main/java/tachyon/client/ReadType.java @@ -26,12 +26,7 @@ public enum ReadType { /** * Read the file and cache it. */ - CACHE(2), - - /** - * Read the file and promote it back to top StorageTier - */ - CACHE_PROMOTE(3); + CACHE(2); private final int mValue; @@ -52,13 +47,6 @@ public int getValue() { * @return true if the read type is CACHE, false otherwise */ public boolean isCache() { - return mValue == CACHE.mValue || mValue == CACHE_PROMOTE.mValue; - } - - /** - * @return true if the read type is CACHE_PROMOTE, false otherwise - */ - public boolean isPromote() { - return mValue == CACHE_PROMOTE.mValue; + return mValue == CACHE.mValue; } } diff --git a/core/src/main/java/tachyon/client/RemoteBlockInStream.java b/core/src/main/java/tachyon/client/RemoteBlockInStream.java index 746d88bb0707..cb2e843ab475 100644 --- a/core/src/main/java/tachyon/client/RemoteBlockInStream.java +++ b/core/src/main/java/tachyon/client/RemoteBlockInStream.java @@ -29,6 +29,7 @@ import tachyon.conf.UserConf; import tachyon.thrift.ClientBlockInfo; import tachyon.thrift.NetAddress; +import tachyon.util.CommonUtils; import tachyon.util.NetworkUtils; import tachyon.worker.nio.DataServerMessage; @@ -237,18 +238,20 @@ public static ByteBuffer readRemoteByteBuffer(TachyonFS tachyonFS, ClientBlockIn List blockLocations = blockInfo.getLocations(); LOG.info("Block locations:" + blockLocations); - for (NetAddress blockLocation : blockLocations) { - String host = blockLocation.mHost; - int port = blockLocation.mSecondaryPort; + for (int k = 0; k < blockLocations.size(); k++) { + String host = blockLocations.get(k).mHost; + int port = blockLocations.get(k).mSecondaryPort; - // The data is not in remote machine's memory if port == -1. + // The data is not in remote machine's memory if port == -1 if (port == -1) { continue; } if (host.equals(InetAddress.getLocalHost().getHostName()) || host.equals(InetAddress.getLocalHost().getHostAddress()) || host.equals(NetworkUtils.getLocalHostName())) { - LOG.warn("Master thinks the local machine has data, But not!"); + String localFileName = + CommonUtils.concat(tachyonFS.getLocalDataFolder(), blockInfo.blockId); + LOG.warn("Master thinks the local machine has data " + localFileName + "! But not!"); } LOG.info(host + ":" + port + " current host is " + NetworkUtils.getLocalHostName() + " " + NetworkUtils.getLocalIpAddress()); @@ -290,8 +293,7 @@ private static ByteBuffer retrieveByteBufferFromRemoteMachine(InetSocketAddress LOG.info("Data " + blockId + " to remote machine " + address + " sent"); - DataServerMessage recvMsg = - DataServerMessage.createBlockResponseMessage(false, blockId, null); + DataServerMessage recvMsg = DataServerMessage.createBlockResponseMessage(false, blockId); while (!recvMsg.isMessageReady()) { int numRead = recvMsg.recv(socketChannel); if (numRead == -1) { diff --git a/core/src/main/java/tachyon/client/TachyonFS.java b/core/src/main/java/tachyon/client/TachyonFS.java index 028e0124e624..a7dc3eab8778 100644 --- a/core/src/main/java/tachyon/client/TachyonFS.java +++ b/core/src/main/java/tachyon/client/TachyonFS.java @@ -23,7 +23,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -36,7 +35,6 @@ import com.google.common.io.Closer; import tachyon.Constants; -import tachyon.StorageDirId; import tachyon.TachyonURI; import tachyon.UnderFileSystem; import tachyon.client.table.RawTable; @@ -46,7 +44,6 @@ import tachyon.thrift.ClientBlockInfo; import tachyon.thrift.ClientDependencyInfo; import tachyon.thrift.ClientFileInfo; -import tachyon.thrift.ClientLocationInfo; import tachyon.thrift.ClientRawTableInfo; import tachyon.thrift.ClientWorkerInfo; import tachyon.util.CommonUtils; @@ -61,7 +58,7 @@ public class TachyonFS extends AbstractTachyonFS { /** * Create a TachyonFS handler. - * + * * @param tachyonPath a Tachyon path contains master address. e.g., tachyon://localhost:19998, * tachyon://localhost:19998/ab/c.txt * @return the corresponding TachyonFS hanlder @@ -113,8 +110,8 @@ public static synchronized TachyonFS get(String masterHost, int masterPort, bool private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); private final long mUserQuotaUnitBytes = UserConf.get().QUOTA_UNIT_BYTES; - private final ExecutorService mExecutorService; private final int mUserFailedSpaceRequestLimits = UserConf.get().FAILED_SPACE_REQUEST_LIMITS; + private final ExecutorService mExecutorService; // The RPC client talks to the system master. private final MasterClient mMasterClient; @@ -135,15 +132,12 @@ public static synchronized TachyonFS get(String masterHost, int masterPort, bool // All Blocks has been locked. private final Map> mLockedBlockIds = new HashMap>(); - // Mapping from block id to id of the StorageDir in which the block is locked - private final Map mLockedBlockIdToLocationInfo = - new HashMap(); // Each user facing block has a unique block lock id. private final AtomicInteger mBlockLockId = new AtomicInteger(0); - // Mapping from Id to Available space of each StorageDir. - private final Map mIdToAvailableSpaceBytes = new HashMap(); + // Available memory space for this client. + private Long mAvailableSpaceBytes; private TachyonFS(TachyonURI tachyonURI) throws IOException { this(new InetSocketAddress(tachyonURI.getHost(), tachyonURI.getPort()), tachyonURI.getScheme() @@ -153,6 +147,7 @@ private TachyonFS(TachyonURI tachyonURI) throws IOException { private TachyonFS(InetSocketAddress masterAddress, boolean zookeeperMode) throws IOException { mMasterAddress = masterAddress; mZookeeperMode = zookeeperMode; + mAvailableSpaceBytes = 0L; mExecutorService = Executors.newFixedThreadPool(2, ThreadFactoryUtils.daemon("client-heartbeat-%d")); @@ -163,15 +158,14 @@ private TachyonFS(InetSocketAddress masterAddress, boolean zookeeperMode) throws } /** - * Update the latest block access time in certain StorageDir on the worker. + * Update the latest block access time on the worker. * - * @param storageDirId the id of the StorageDir which contains the block * @param blockId the local block's id * @throws IOException */ - synchronized void accessLocalBlock(long storageDirId, long blockId) throws IOException { + synchronized void accessLocalBlock(long blockId) throws IOException { if (mWorkerClient.isLocal()) { - mWorkerClient.accessBlock(storageDirId, blockId); + mWorkerClient.accessBlock(blockId); } } @@ -199,12 +193,11 @@ synchronized boolean asyncCheckpoint(int fid) throws IOException { /** * Notify the worker the block is cached. * - * @param storageDirId the id of the StorageDir which contains the block * @param blockId the block id * @throws IOException */ - public synchronized void cacheBlock(long storageDirId, long blockId) throws IOException { - mWorkerClient.cacheBlock(storageDirId, blockId); + public synchronized void cacheBlock(long blockId) throws IOException { + mWorkerClient.cacheBlock(blockId); } /** @@ -215,13 +208,7 @@ public synchronized void cacheBlock(long storageDirId, long blockId) throws IOEx @Override public synchronized void close() throws IOException { if (mWorkerClient.isConnected()) { - for (Entry availableSpace : mIdToAvailableSpaceBytes.entrySet()) { - if (availableSpace.getValue() > 0) { - mWorkerClient.returnSpace(mMasterClient.getUserId(), availableSpace.getKey(), - availableSpace.getValue()); - } - } - mWorkerClient.close(); + mWorkerClient.returnSpace(mMasterClient.getUserId(), mAvailableSpaceBytes); } try { mCloser.close(); @@ -241,45 +228,33 @@ synchronized void completeFile(int fid) throws IOException { } /** - * Create a user local temporary folder in some StorageDir + * Create a user local temporary folder and return it * - * @param the id of the StorageDir - * @return the user local temporary folder + * @return the local temporary folder for the user or null if unable to allocate one. * @throws IOException */ - public synchronized String createAndGetUserLocalTempFolder(long storageDirId) - throws IOException { - String userTempFolder = mWorkerClient.getUserLocalTempFolder(storageDirId); - return createAndGetUserLocalTempFolder(userTempFolder); - } + synchronized File createAndGetUserLocalTempFolder() throws IOException { + String userTempFolder = mWorkerClient.getUserTempFolder(); - /** - * Create a user local temporary folder specified by the input path - * - * @param the path of the temporary folder - * @return the path of the temporary folder - * @throws IOException - */ - public synchronized String createAndGetUserLocalTempFolder(String userTempFolder) - throws IOException { - if (StringUtils.isBlank(userTempFolder) || userTempFolder == null) { + if (StringUtils.isBlank(userTempFolder)) { LOG.error("Unable to get local temporary folder \"{}\" for user.", userTempFolder); return null; } - File tempFolder = new File(userTempFolder); - if (!tempFolder.exists()) { - if (tempFolder.mkdir()) { - CommonUtils.changeLocalFileToFullPermission(tempFolder.getAbsolutePath()); - LOG.info("Folder " + tempFolder + " was created!"); + File ret = new File(userTempFolder); + if (!ret.exists()) { + if (ret.mkdir()) { + CommonUtils.changeLocalFileToFullPermission(ret.getAbsolutePath()); + LOG.info("Folder " + ret + " was created!"); } else { - LOG.error("Failed to create folder " + tempFolder); + LOG.error("Failed to create folder " + ret); return null; } } - return tempFolder.getAbsolutePath(); + return ret; } + /** * Create a user UnderFileSystem temporary folder and return it * @@ -344,7 +319,7 @@ public synchronized int createFile(TachyonURI path, TachyonURI ufsPath, long blo /** * Create a file with the default block size (1GB) in the system. It also creates necessary * folders along the path. // TODO It should not create necessary path. - * + * * @param path the path of the file * @return The unique file id. It returns -1 if the creation failed. * @throws IOException If file already exists, or path is invalid. @@ -388,7 +363,7 @@ public synchronized int createRawTable(TachyonURI path, int columns, ByteBuffer /** * Deletes a file or folder - * + * * @param fileId The id of the file / folder. If it is not -1, path parameter is ignored. * Otherwise, the method uses the path parameter. * @param path The path of the file / folder. It could be empty iff id is not -1. @@ -406,7 +381,7 @@ public synchronized boolean delete(int fileId, TachyonURI path, boolean recursiv /** * Delete the file denoted by the path. - * + * * @param path the file path * @param recursive if delete the path recursively. * @return true if the deletion succeed (including the case that the path does not exist in the @@ -526,7 +501,7 @@ public synchronized TachyonFile getFile(TachyonURI path) throws IOException { /** * Get TachyonFile based on the path. Does not utilize the file metadata cache. - * + * * @param path file path. * @return TachyonFile of the path, or null if the file does not exist. * @throws IOException @@ -660,48 +635,6 @@ public synchronized ClientFileInfo getFileStatus(int fileId, TachyonURI path, return info; } - /** - * Get space from available space of StorageDirs - * - * @param requestSpaceBytes size to request in bytes - * @return the id of the StorageDir allocated - * @throws IOException - */ - private synchronized long getFromAvaliableSpace(long requestSpaceBytes) throws IOException { - long storageDirId = StorageDirId.unknownId(); - for (Entry entry : mIdToAvailableSpaceBytes.entrySet()) { - if (entry.getValue() >= requestSpaceBytes - && StorageDirId.compareStorageLevel(entry.getKey(), storageDirId) < 0) { - storageDirId = entry.getKey(); - } - } - if (!StorageDirId.isUnknown(storageDirId)) { - long availableSpaceBytes = mIdToAvailableSpaceBytes.get(storageDirId); - mIdToAvailableSpaceBytes.put(storageDirId, availableSpaceBytes - requestSpaceBytes); - } - return storageDirId; - } - - /** - * Get space from available space of specific StorageDir for the user - * - * @param storageDirId the id of the StorageDir - * @param requestSpaceBytes size of the space to request in bytes - * @return true if success, false otherwise - * @throws IOException - */ - private synchronized boolean getFromAvaliableSpace(long storageDirId, long requestSpaceBytes) - throws IOException { - if (mIdToAvailableSpaceBytes.containsKey(storageDirId)) { - long availableSpaceBytes = mIdToAvailableSpaceBytes.get(storageDirId); - if (availableSpaceBytes >= requestSpaceBytes) { - mIdToAvailableSpaceBytes.put(storageDirId, availableSpaceBytes - requestSpaceBytes); - return true; - } - } - return false; - } - /** * Get the RawTable by id * @@ -729,14 +662,11 @@ public synchronized RawTable getRawTable(TachyonURI path) throws IOException { } /** - * Get location information of the block file on local worker - * - * @param blockId id of the block - * @return the location information of the block file on local worker + * @return the local root data folder * @throws IOException */ - synchronized ClientLocationInfo getLocalBlockLocation(long blockId) throws IOException { - return mWorkerClient.getLocalBlockLocation(blockId); + synchronized String getLocalDataFolder() throws IOException { + return mWorkerClient.getDataFolder(); } /** @@ -756,7 +686,7 @@ public synchronized TachyonURI getUri() { String authority = mMasterAddress.getHostName() + ":" + mMasterAddress.getPort(); return new TachyonURI(scheme, authority, TachyonURI.SEPARATOR); } - + /** * Returns the userId of the master client. This is only used for testing. * @@ -813,39 +743,32 @@ public synchronized List listStatus(TachyonURI path) throws IOEx } /** - * Lock a block in certain StorageDir in the current TachyonFS. + * Lock a block in the current TachyonFS. * * @param blockId The id of the block to lock. blockId must be positive. * @param blockLockId The block lock id of the block of lock. blockLockId must be * non-negative. - * @return the Id of the StorageDir in which the block is locked - * @throws IOException + * @return true if successfully lock the block, false otherwise (or invalid parameter). */ - synchronized ClientLocationInfo lockBlock(long blockId, int blockLockId) - throws IOException { + synchronized boolean lockBlock(long blockId, int blockLockId) throws IOException { if (blockId <= 0 || blockLockId < 0) { - return null; + return false; } if (mLockedBlockIds.containsKey(blockId)) { mLockedBlockIds.get(blockId).add(blockLockId); - return mLockedBlockIdToLocationInfo.get(blockId); + return true; } if (!mWorkerClient.isLocal()) { - return null; - } - ClientLocationInfo locationInfo = - mWorkerClient.lockBlock(blockId, mMasterClient.getUserId()); - - if (locationInfo != null) { - Set lockIds = new HashSet(4); - lockIds.add(blockLockId); - mLockedBlockIds.put(blockId, lockIds); - mLockedBlockIdToLocationInfo.put(blockId, locationInfo); - return locationInfo; + return false; } - return null; + mWorkerClient.lockBlock(blockId, mMasterClient.getUserId()); + + Set lockIds = new HashSet(4); + lockIds.add(blockLockId); + mLockedBlockIds.put(blockId, lockIds); + return true; } /** @@ -884,33 +807,8 @@ public synchronized boolean freepath(int fileId, TachyonURI path, boolean recurs return mMasterClient.user_freepath(fileId, path.getPath(), recursive); } - /** - * Promote block file back to the top StorageTier, after the block file is accessed. - * - * @param blockId the id of the block - * @return true if success, false otherwise - * @throws IOException - */ - public synchronized boolean promoteBlock(long blockId) throws IOException { - if (mWorkerClient.isLocal()) { - return mWorkerClient.promoteBlock(mMasterClient.getUserId(), blockId); - } - return false; - } - - /** - * Release space to some StorageDir. - * - * @param storageDirId the id of the StorageDir which the space will be release to - * @param releaseSpaceBytes the size of the space to be released in bytes - */ - public synchronized void releaseSpace(long storageDirId, long releaseSpaceBytes) { - if (mIdToAvailableSpaceBytes.containsKey(storageDirId)) { - long availableSpaceBytes = mIdToAvailableSpaceBytes.get(storageDirId); - mIdToAvailableSpaceBytes.put(storageDirId, availableSpaceBytes + releaseSpaceBytes); - } else { - LOG.warn("Unknown StorageDir! ID:" + storageDirId); - } + public synchronized void releaseSpace(long releaseSpaceBytes) { + mAvailableSpaceBytes += releaseSpaceBytes; } /** @@ -954,74 +852,36 @@ public synchronized void requestFilesInDependency(int depId) throws IOException /** * Try to request space from worker. Only works when a local worker exists. * - * @param requestSpaceBytes size to request in bytes - * @return the id of the StorageDir that space is allocated in + * @param requestSpaceBytes the space size in bytes + * @return true if succeed, false otherwise * @throws IOException */ - public synchronized ClientLocationInfo requestSpace(long requestSpaceBytes) throws IOException { - if (!hasLocalWorker()) { - return null; + public synchronized boolean requestSpace(long requestSpaceBytes) throws IOException { + if (!mWorkerClient.isLocal()) { + return false; } - long storageDirId = getFromAvaliableSpace(requestSpaceBytes); - if (storageDirId != StorageDirId.unknownId()) { - String path = mWorkerClient.getUserLocalTempFolder(storageDirId); - return new ClientLocationInfo(storageDirId, path); - } else { - for (int attempt = 0; attempt < mUserFailedSpaceRequestLimits; attempt ++) { - long toRequestSpaceBytes = Math.max(requestSpaceBytes, mUserQuotaUnitBytes); - ClientLocationInfo locationInfo = - mWorkerClient.requestSpace(mMasterClient.getUserId(), toRequestSpaceBytes); - if (locationInfo == null) { - continue; - } - storageDirId = locationInfo.getStorageDirId(); - Long availableSpaceBytes = mIdToAvailableSpaceBytes.get(storageDirId); - if (availableSpaceBytes != null) { - availableSpaceBytes += toRequestSpaceBytes; - } else { - availableSpaceBytes = toRequestSpaceBytes; + int failedTimes = 0; + while (mAvailableSpaceBytes < requestSpaceBytes) { + long toRequestSpaceBytes = + Math.max(requestSpaceBytes - mAvailableSpaceBytes, mUserQuotaUnitBytes); + if (mWorkerClient.requestSpace(mMasterClient.getUserId(), toRequestSpaceBytes)) { + mAvailableSpaceBytes += toRequestSpaceBytes; + } else { + LOG.info("Failed to request " + toRequestSpaceBytes + " bytes local space. " + "Time " + + (failedTimes ++)); + if (failedTimes == mUserFailedSpaceRequestLimits) { + return false; } - mIdToAvailableSpaceBytes.put(storageDirId, availableSpaceBytes - requestSpaceBytes); - return locationInfo; } - return null; } - } - /** - * Try to request space from certain StorageDir on worker. Only works when a local worker exists. - * - * @param storageDirId the id of the StorageDir that space will be allocated in - * @param requestSpaceBytes size to request in bytes - * @return true if success, false otherwise - * @throws IOException - */ - public synchronized boolean requestSpace(long storageDirId, long requestSpaceBytes) - throws IOException { - if (!hasLocalWorker()) { - return false; - } - if (getFromAvaliableSpace(storageDirId, requestSpaceBytes)) { - return true; - } else { - long availableSpaceBytes = 0; - if (mIdToAvailableSpaceBytes.containsKey(storageDirId)) { - availableSpaceBytes = mIdToAvailableSpaceBytes.get(storageDirId); - } - for (int attempt = 0; attempt < mUserFailedSpaceRequestLimits; attempt ++) { - long toRequestSpaceBytes = - Math.max(requestSpaceBytes - availableSpaceBytes, mUserQuotaUnitBytes); - boolean reqResult = - mWorkerClient.requestSpace(mMasterClient.getUserId(), storageDirId, - toRequestSpaceBytes); - if (reqResult) { - availableSpaceBytes += toRequestSpaceBytes; - mIdToAvailableSpaceBytes.put(storageDirId, availableSpaceBytes - requestSpaceBytes); - return true; - } - } + if (mAvailableSpaceBytes < requestSpaceBytes) { return false; } + + mAvailableSpaceBytes -= requestSpaceBytes; + + return true; } /** @@ -1051,15 +911,16 @@ public String toString() { * @param blockId The id of the block to unlock. blockId must be positive. * @param blockLockId The block lock id of the block of unlock. blockLockId must be * non-negative. + * @return true if successfully unlock the block with blockLockId, false otherwise + * (or invalid parameter). */ - synchronized boolean unlockBlock(long blockId, int blockLockId) - throws IOException { + synchronized boolean unlockBlock(long blockId, int blockLockId) throws IOException { if (blockId <= 0 || blockLockId < 0) { return false; } if (!mLockedBlockIds.containsKey(blockId)) { - return false; + return true; } Set lockIds = mLockedBlockIds.get(blockId); lockIds.remove(blockLockId); @@ -1071,9 +932,10 @@ synchronized boolean unlockBlock(long blockId, int blockLockId) return false; } - mLockedBlockIdToLocationInfo.remove(blockId); + mWorkerClient.unlockBlock(blockId, mMasterClient.getUserId()); mLockedBlockIds.remove(blockId); - return mWorkerClient.unlockBlock(blockId, mMasterClient.getUserId()); + + return true; } /** Alias for setPinned(fid, false). */ diff --git a/core/src/main/java/tachyon/client/TachyonFile.java b/core/src/main/java/tachyon/client/TachyonFile.java index a260e8174a27..a438eafd5db0 100644 --- a/core/src/main/java/tachyon/client/TachyonFile.java +++ b/core/src/main/java/tachyon/client/TachyonFile.java @@ -13,6 +13,7 @@ package tachyon.client; +import java.io.File; import java.io.FileNotFoundException; import java.io.InputStream; import java.io.IOException; @@ -33,8 +34,8 @@ import tachyon.conf.UserConf; import tachyon.thrift.ClientBlockInfo; import tachyon.thrift.ClientFileInfo; -import tachyon.thrift.ClientLocationInfo; import tachyon.thrift.NetAddress; +import tachyon.util.CommonUtils; /** * Tachyon File. @@ -176,12 +177,16 @@ public InStream getInStream(ReadType readType) throws IOException { */ public String getLocalFilename(int blockIndex) throws IOException { ClientBlockInfo blockInfo = getClientBlockInfo(blockIndex); - ClientLocationInfo blockLocation = mTachyonFS.getLocalBlockLocation(blockInfo.getBlockId()); - if (blockLocation == null) { - return null; - } else { - return blockLocation.getPath(); + + String rootFolder = mTachyonFS.getLocalDataFolder(); + if (rootFolder != null) { + String localFileName = CommonUtils.concat(rootFolder, blockInfo.getBlockId()); + File file = new File(localFileName); + if (file.exists()) { + return localFileName; + } } + return null; } /** @@ -327,18 +332,6 @@ public boolean needPin() throws IOException { return mTachyonFS.getFileStatus(mFileId, false).isPinned; } - /** - * Promote block after access it - * - * @param blockIndex the index of the block - * @return true if success, false otherwise - * @throws IOException - */ - public boolean promoteBlock(int blockIndex) throws IOException { - ClientBlockInfo blockInfo = getClientBlockInfo(blockIndex); - return mTachyonFS.promoteBlock(blockInfo.getBlockId()); - } - /** * Advanced API. * @@ -395,20 +388,16 @@ private TachyonByteBuffer readLocalByteBuffer(int blockIndex, long offset, long long blockId = info.blockId; int blockLockId = mTachyonFS.getBlockLockId(); - ClientLocationInfo locationInfo = mTachyonFS.lockBlock(blockId, blockLockId); - if (locationInfo == null) { + if (!mTachyonFS.lockBlock(blockId, blockLockId)) { return null; } - + String localFileName = getLocalFilename(blockIndex); Closer closer = Closer.create(); - String localFileName = locationInfo.getPath(); if (localFileName != null) { try { - RandomAccessFile localFile; - long fileLength; - localFile = closer.register(new RandomAccessFile(localFileName, "r")); - fileLength = localFile.length(); + RandomAccessFile localFile = closer.register(new RandomAccessFile(localFileName, "r")); + long fileLength = localFile.length(); String error = null; if (offset > fileLength) { error = String.format("Offset(%d) is larger than file length(%d)", offset, fileLength); @@ -428,7 +417,7 @@ private TachyonByteBuffer readLocalByteBuffer(int blockIndex, long offset, long FileChannel localFileChannel = closer.register(localFile.getChannel()); final ByteBuffer buf = localFileChannel.map(FileChannel.MapMode.READ_ONLY, offset, len); - mTachyonFS.accessLocalBlock(locationInfo.getStorageDirId(), blockId); + mTachyonFS.accessLocalBlock(blockId); return new TachyonByteBuffer(mTachyonFS, buf, blockId, blockLockId); } catch (FileNotFoundException e) { LOG.info(localFileName + " is not on local disk."); diff --git a/core/src/main/java/tachyon/conf/WorkerConf.java b/core/src/main/java/tachyon/conf/WorkerConf.java index d344ffff9df4..0f22cabf0a1f 100644 --- a/core/src/main/java/tachyon/conf/WorkerConf.java +++ b/core/src/main/java/tachyon/conf/WorkerConf.java @@ -82,11 +82,11 @@ public static synchronized WorkerConf get() { public final EvictStrategyType EVICT_STRATEGY_TYPE; public final AllocateStrategyType ALLOCATE_STRATEGY_TYPE; - public final int MAX_HIERARCHY_STORAGE_LEVEL; public final StorageLevelAlias[] STORAGE_LEVEL_ALIAS; public final String[] STORAGE_TIER_DIRS; public final String[] STORAGE_TIER_DIR_QUOTA; + public final String[] STORAGE_TIER_DIR_QUOTA_DEFAULTS = "512MB,64GB,1TB".split(","); private WorkerConf() { MASTER_HOSTNAME = getProperty("tachyon.master.hostname", NetworkUtils.getLocalHostName()); @@ -95,7 +95,7 @@ private WorkerConf() { PORT = getIntProperty("tachyon.worker.port", Constants.DEFAULT_WORKER_PORT); DATA_PORT = getIntProperty("tachyon.worker.data.port", Constants.DEFAULT_WORKER_DATA_SERVER_PORT); - DATA_FOLDER = getProperty("tachyon.worker.data.folder", "/datastore"); + DATA_FOLDER = getProperty("tachyon.worker.data.folder", "/mnt/ramdisk"); MEMORY_SIZE = CommonUtils.parseSpaceSize(getProperty("tachyon.worker.memory.size", (128 * Constants.MB) + "")); @@ -146,17 +146,16 @@ private WorkerConf() { STORAGE_LEVEL_ALIAS[i] = getEnumProperty("tachyon.worker.hierarchystore.level" + i + ".alias", StorageLevelAlias.MEM); - if (STORAGE_LEVEL_ALIAS[i].equals(StorageLevelAlias.MEM)) { + STORAGE_TIER_DIRS[i] = + getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.path", "/mnt/ramdisk"); + if (i < STORAGE_TIER_DIR_QUOTA_DEFAULTS.length) { STORAGE_TIER_DIR_QUOTA[i] = getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.quota", - MEMORY_SIZE + ""); - STORAGE_TIER_DIRS[i] = - getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.path", "/mnt/ramdisk"); + STORAGE_TIER_DIR_QUOTA_DEFAULTS[i]); } else { STORAGE_TIER_DIR_QUOTA[i] = - getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.quota"); - STORAGE_TIER_DIRS[i] = - getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.path"); + getProperty("tachyon.worker.hierarchystore.level" + i + ".dirs.quota", + STORAGE_TIER_DIR_QUOTA_DEFAULTS[STORAGE_TIER_DIR_QUOTA_DEFAULTS.length - 1]); } } } diff --git a/core/src/main/java/tachyon/master/BlockInfo.java b/core/src/main/java/tachyon/master/BlockInfo.java index 0967ec4160ff..a4b7394ae0ce 100644 --- a/core/src/main/java/tachyon/master/BlockInfo.java +++ b/core/src/main/java/tachyon/master/BlockInfo.java @@ -23,8 +23,6 @@ import java.util.Map; import tachyon.Pair; -import tachyon.StorageDirId; -import tachyon.StorageLevelAlias; import tachyon.UnderFileSystem; import tachyon.thrift.ClientBlockInfo; import tachyon.thrift.NetAddress; @@ -74,8 +72,7 @@ public static int computeInodeId(long blockId) { public final long mOffset; public final long mLength; - private final Map mLocations = new HashMap(5); - private final Map mStorageDirIds = new HashMap(5); + private Map mLocations = new HashMap(5); /** * @param inodeFile @@ -95,11 +92,9 @@ public static int computeInodeId(long blockId) { * * @param workerId The id of the worker * @param workerAddress The net address of the worker - * @param storageDirId The id of the StorageDir which block is located in */ - public synchronized void addLocation(long workerId, NetAddress workerAddress, long storageDirId) { + public synchronized void addLocation(long workerId, NetAddress workerAddress) { mLocations.put(workerId, workerAddress); - mStorageDirIds.put(workerAddress, storageDirId); } /** @@ -114,7 +109,6 @@ public synchronized ClientBlockInfo generateClientBlockInfo() { ret.offset = mOffset; ret.length = mLength; ret.locations = getLocations(); - ret.storageDirIds = mStorageDirIds; return ret; } @@ -178,13 +172,7 @@ public synchronized List getLocations() { * @return true if the block is in some worker's memory, false otherwise */ public synchronized boolean isInMemory() { - for (long storageDirId : mStorageDirIds.values()) { - int storageLevelValue = StorageDirId.getStorageLevelAliasValue(storageDirId); - if (storageLevelValue == StorageLevelAlias.MEM.getValue()) { - return true; - } - } - return false; + return mLocations.size() > 0; } /** @@ -193,9 +181,7 @@ public synchronized boolean isInMemory() { * @param workerId The id of the removed worker */ public synchronized void removeLocation(long workerId) { - if (mLocations.containsKey(workerId)) { - mStorageDirIds.remove(mLocations.remove(workerId)); - } + mLocations.remove(workerId); } @Override diff --git a/core/src/main/java/tachyon/master/InodeFile.java b/core/src/main/java/tachyon/master/InodeFile.java index 3a08e00aade2..0340535f99c2 100644 --- a/core/src/main/java/tachyon/master/InodeFile.java +++ b/core/src/main/java/tachyon/master/InodeFile.java @@ -76,7 +76,7 @@ static InodeFile loadImage(ImageElement ele) throws IOException { private boolean mCache = false; private String mUfsPath = ""; - private final List mBlocks = new ArrayList(3); + private List mBlocks = new ArrayList(3); private int mDependencyId; @@ -133,15 +133,14 @@ public synchronized void addBlock(BlockInfo blockInfo) throws BlockInfoException * @param blockIndex The index of the block in the file * @param workerId The id of the worker * @param workerAddress The net address of the worker - * @param storageDirId The id of the StorageDir which block is located in * @throws BlockInfoException */ - public synchronized void addLocation(int blockIndex, long workerId, NetAddress workerAddress, - long storageDirId) throws BlockInfoException { + public synchronized void addLocation(int blockIndex, long workerId, NetAddress workerAddress) + throws BlockInfoException { if (blockIndex < 0 || blockIndex >= mBlocks.size()) { throw new BlockInfoException("BlockIndex " + blockIndex + " out of bounds." + toString()); } - mBlocks.get(blockIndex).addLocation(workerId, workerAddress, storageDirId); + mBlocks.get(blockIndex).addLocation(workerId, workerAddress); } @Override @@ -375,8 +374,7 @@ public synchronized boolean isFullyInMemory() { * @param workerId The id of the removed location worker * @throws BlockInfoException */ - public synchronized void removeLocation(int blockIndex, long workerId) - throws BlockInfoException { + public synchronized void removeLocation(int blockIndex, long workerId) throws BlockInfoException { if (blockIndex < 0 || blockIndex >= mBlocks.size()) { throw new BlockInfoException("BlockIndex " + blockIndex + " out of bounds." + toString()); } diff --git a/core/src/main/java/tachyon/master/MasterClient.java b/core/src/main/java/tachyon/master/MasterClient.java index bf02bdb99cfe..073bd805215c 100644 --- a/core/src/main/java/tachyon/master/MasterClient.java +++ b/core/src/main/java/tachyon/master/MasterClient.java @@ -22,7 +22,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -81,7 +80,7 @@ public final class MasterClient implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); private static final int MAX_CONNECT_TRY = CommonConf.get().MASTER_RETRY_COUNT; - private final boolean mUseZookeeper; + private boolean mUseZookeeper; private MasterService.Client mClient = null; private InetSocketAddress mMasterAddress = null; private TProtocol mProtocol = null; @@ -732,14 +731,14 @@ public synchronized boolean user_freepath(int fileId, String path, boolean recur return false; } - public synchronized void worker_cacheBlock(long workerId, long workerUsedBytes, - long storageDirId, long blockId, long length) throws IOException, FileDoesNotExistException, - SuspectedFileSizeException, BlockInfoException { + public synchronized void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, + long length) throws IOException, FileDoesNotExistException, SuspectedFileSizeException, + BlockInfoException { while (!mIsShutdown) { connect(); try { - mClient.worker_cacheBlock(workerId, workerUsedBytes, storageDirId, blockId, length); + mClient.worker_cacheBlock(workerId, workerUsedBytes, blockId, length); return; } catch (FileDoesNotExistException e) { throw e; @@ -782,13 +781,12 @@ public synchronized List worker_getPriorityDependencyList() throws IOEx } public synchronized Command worker_heartbeat(long workerId, long usedBytes, - List removedBlockIds, Map> addedBlockIds) - throws IOException { + List removedPartitionList) throws IOException { while (!mIsShutdown) { connect(); try { - return mClient.worker_heartbeat(workerId, usedBytes, removedBlockIds, addedBlockIds); + return mClient.worker_heartbeat(workerId, usedBytes, removedPartitionList); } catch (BlockInfoException e) { throw new IOException(e); } catch (TException e) { @@ -815,8 +813,7 @@ public synchronized Command worker_heartbeat(long workerId, long usedBytes, * @throws TException */ public synchronized long worker_register(NetAddress workerNetAddress, long totalBytes, - long usedBytes, Map> currentBlockList) - throws BlockInfoException, IOException { + long usedBytes, List currentBlockList) throws BlockInfoException, IOException { while (!mIsShutdown) { connect(); diff --git a/core/src/main/java/tachyon/master/MasterInfo.java b/core/src/main/java/tachyon/master/MasterInfo.java index c2d75fcdcdd7..d2d8ce51a5fd 100644 --- a/core/src/main/java/tachyon/master/MasterInfo.java +++ b/core/src/main/java/tachyon/master/MasterInfo.java @@ -45,9 +45,7 @@ import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Optional; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import tachyon.Constants; import tachyon.HeartbeatExecutor; @@ -218,7 +216,7 @@ public void run() { for (String cmd : cmds) { String filePath = CommonConf.get().TACHYON_HOME + "/logs/rerun-" + mRerunCounter.incrementAndGet(); - // TODO use bounded threads (ExecutorService) + //TODO use bounded threads (ExecutorService) Thread thread = new Thread(new RecomputeCommand(cmd, filePath)); thread.setName("recompute-command-" + cmd); thread.start(); @@ -462,7 +460,7 @@ int _createDependency(List parentsIds, List childrenIds, Strin // TODO Make this API better. /** * Internal API. - * + * * @param recursive If recursive is true and the filesystem tree is not filled in all the way to * path yet, it fills in the missing components. * @param path The path to create @@ -664,7 +662,7 @@ boolean _delete(int fileId, boolean recursive, long opTimeMs) throws TachyonExce /** * Get the raw table info associated with the given id. - * + * * @param path The path of the table * @param inode The inode at the path * @return the table info @@ -687,7 +685,7 @@ public ClientRawTableInfo _getClientRawTableInfo(TachyonURI path, Inode inode) /** * Get the names of the sub-directories at the given path. - * + * * @param inode The inode to list * @param path The path of the given inode * @param recursive If true, recursively add the paths of the sub-directories @@ -743,7 +741,7 @@ void _recomputePinnedFiles(Inode inode, Optional setPinState, long opTi /** * Rename a file to the given path, inner method. - * + * * @param fileId The id of the file to rename * @param dstPath The new path of the file * @param opTimeMs The time of the rename operation, in milliseconds @@ -908,8 +906,7 @@ private void addToInodeMap(Inode inode, Map map) { * @throws SuspectedFileSizeException * @throws BlockInfoException */ - public int cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, - long length) + public int cacheBlock(long workerId, long workerUsedBytes, long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException { LOG.debug("Cache block: {}", CommonUtils.parametersToString(workerId, workerUsedBytes, blockId, length)); @@ -936,7 +933,7 @@ public int cacheBlock(long workerId, long workerUsedBytes, long storageDirId, lo addBlock(tFile, new BlockInfo(tFile, blockIndex, length), System.currentTimeMillis()); } - tFile.addLocation(blockIndex, workerId, tWorkerInfo.mWorkerAddress, storageDirId); + tFile.addLocation(blockIndex, workerId, tWorkerInfo.mWorkerAddress); if (tFile.hasCheckpointed()) { return -1; @@ -1995,12 +1992,12 @@ private void recomputePinnedFiles(Inode inode, Optional setPinState) { * @param workerNetAddress The address of the worker to register * @param totalBytes The capacity of the worker in bytes * @param usedBytes The number of bytes already used in the worker - * @param currentBlockIds Mapping from id of the StorageDir to id list of the blocks + * @param currentBlockIds The id's of the blocks held by the worker * @return the new id of the registered worker * @throws BlockInfoException */ public long registerWorker(NetAddress workerNetAddress, long totalBytes, long usedBytes, - Map> currentBlockIds) throws BlockInfoException { + List currentBlockIds) throws BlockInfoException { long id = 0; NetAddress workerAddress = new NetAddress(workerNetAddress); LOG.info("registerWorker(): WorkerNetAddress: " + workerAddress); @@ -2020,9 +2017,7 @@ public long registerWorker(NetAddress workerNetAddress, long totalBytes, long us id = mStartTimeNSPrefix + mWorkerCounter.incrementAndGet(); MasterWorkerInfo tWorkerInfo = new MasterWorkerInfo(id, workerAddress, totalBytes); tWorkerInfo.updateUsedBytes(usedBytes); - for (List blockIds : currentBlockIds.values()) { - tWorkerInfo.updateBlocks(true, blockIds); - } + tWorkerInfo.updateBlocks(true, currentBlockIds); tWorkerInfo.updateLastUpdatedTimeMs(); mWorkers.put(id, tWorkerInfo); mWorkerAddressToId.put(workerAddress, id); @@ -2030,17 +2025,14 @@ public long registerWorker(NetAddress workerNetAddress, long totalBytes, long us } synchronized (mRootLock) { - for (Entry> blockIds : currentBlockIds.entrySet()) { - long storageDirId = blockIds.getKey(); - for (long blockId : blockIds.getValue()) { - int fileId = BlockInfo.computeInodeId(blockId); - int blockIndex = BlockInfo.computeBlockIndex(blockId); - Inode inode = mFileIdToInodes.get(fileId); - if (inode != null && inode.isFile()) { - ((InodeFile) inode).addLocation(blockIndex, id, workerAddress, storageDirId); - } else { - LOG.warn("registerWorker failed to add fileId " + fileId + " blockIndex " + blockIndex); - } + for (long blockId : currentBlockIds) { + int fileId = BlockInfo.computeInodeId(blockId); + int blockIndex = BlockInfo.computeBlockIndex(blockId); + Inode inode = mFileIdToInodes.get(fileId); + if (inode != null && inode.isFile()) { + ((InodeFile) inode).addLocation(blockIndex, id, workerAddress); + } else { + LOG.warn("registerWorker failed to add fileId " + fileId + " blockIndex " + blockIndex); } } } @@ -2331,13 +2323,11 @@ public void updateRawTableMetadata(int tableId, ByteBuffer metadata) * * @param workerId The id of the worker to deal with * @param usedBytes The number of bytes used in the worker - * @param removedBlockIds The list of removed block ids - * @param evictedBlockIds Mapping from id of the StorageDir and id list of blocks evicted in + * @param removedBlockIds The id's of the blocks that have been removed * @return a command specifying an action to take * @throws BlockInfoException */ - public Command workerHeartbeat(long workerId, long usedBytes, List removedBlockIds, - Map> evictedBlockIds) + public Command workerHeartbeat(long workerId, long usedBytes, List removedBlockIds) throws BlockInfoException { LOG.debug("WorkerId: {}", workerId); synchronized (mRootLock) { @@ -2368,27 +2358,6 @@ public Command workerHeartbeat(long workerId, long usedBytes, List removed } } - for (Entry> addedBlocks : evictedBlockIds.entrySet()) { - long storageDirId = addedBlocks.getKey(); - for (long blockId : addedBlocks.getValue()) { - int fileId = BlockInfo.computeInodeId(blockId); - int blockIndex = BlockInfo.computeBlockIndex(blockId); - Inode inode = mFileIdToInodes.get(fileId); - if (inode == null) { - LOG.error("File " + fileId + " does not exist"); - } else if (inode.isFile()) { - List blockInfoList = ((InodeFile) inode).getBlockList(); - NetAddress workerAddress = mWorkers.get(workerId).getAddress(); - if (blockInfoList.size() <= blockIndex) { - throw new BlockInfoException("BlockInfo not found! blockIndex:" + blockIndex); - } else { - BlockInfo blockInfo = blockInfoList.get(blockIndex); - blockInfo.addLocation(workerId, workerAddress, storageDirId); - } - } - } - } - List toRemovedBlocks = tWorkerInfo.getToRemovedBlocks(); if (toRemovedBlocks.size() != 0) { return new Command(CommandType.Free, toRemovedBlocks); diff --git a/core/src/main/java/tachyon/master/MasterServiceHandler.java b/core/src/main/java/tachyon/master/MasterServiceHandler.java index 8ba52f99d9c5..a99f83389aa5 100644 --- a/core/src/main/java/tachyon/master/MasterServiceHandler.java +++ b/core/src/main/java/tachyon/master/MasterServiceHandler.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; import org.apache.thrift.TException; @@ -306,10 +305,9 @@ public void user_updateRawTableMetadata(int tableId, ByteBuffer metadata) } @Override - public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, - long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, - BlockInfoException, TException { - mMasterInfo.cacheBlock(workerId, workerUsedBytes, storageDirId, blockId, length); + public void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length) + throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, TException { + mMasterInfo.cacheBlock(workerId, workerUsedBytes, blockId, length); } @Override @@ -324,15 +322,14 @@ public List worker_getPriorityDependencyList() throws TException { } @Override - public Command worker_heartbeat(long workerId, long usedBytes, - List removedBlockIds, Map> evictedBlockIds) + public Command worker_heartbeat(long workerId, long usedBytes, List removedBlockIds) throws BlockInfoException, TException { - return mMasterInfo.workerHeartbeat(workerId, usedBytes, removedBlockIds, evictedBlockIds); + return mMasterInfo.workerHeartbeat(workerId, usedBytes, removedBlockIds); } @Override public long worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, - Map> currentBlockIds) throws BlockInfoException, TException { + List currentBlockIds) throws BlockInfoException, TException { return mMasterInfo.registerWorker(workerNetAddress, totalBytes, usedBytes, currentBlockIds); } } diff --git a/core/src/main/java/tachyon/thrift/ClientBlockInfo.java b/core/src/main/java/tachyon/thrift/ClientBlockInfo.java index 20d91890be7f..e5b27fd89a60 100644 --- a/core/src/main/java/tachyon/thrift/ClientBlockInfo.java +++ b/core/src/main/java/tachyon/thrift/ClientBlockInfo.java @@ -39,7 +39,6 @@ public class ClientBlockInfo implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -51,15 +50,13 @@ public class ClientBlockInfo implements org.apache.thrift.TBase locations; // required - public Map storageDirIds; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { BLOCK_ID((short)1, "blockId"), OFFSET((short)2, "offset"), LENGTH((short)3, "length"), - LOCATIONS((short)4, "locations"), - STORAGE_DIR_IDS((short)5, "storageDirIds"); + LOCATIONS((short)4, "locations"); private static final Map byName = new HashMap(); @@ -82,8 +79,6 @@ public static _Fields findByThriftId(int fieldId) { return LENGTH; case 4: // LOCATIONS return LOCATIONS; - case 5: // STORAGE_DIR_IDS - return STORAGE_DIR_IDS; default: return null; } @@ -140,10 +135,6 @@ public String getFieldName() { tmpMap.put(_Fields.LOCATIONS, new org.apache.thrift.meta_data.FieldMetaData("locations", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NetAddress.class)))); - tmpMap.put(_Fields.STORAGE_DIR_IDS, new org.apache.thrift.meta_data.FieldMetaData("storageDirIds", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NetAddress.class), - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClientBlockInfo.class, metaDataMap); } @@ -155,8 +146,7 @@ public ClientBlockInfo( long blockId, long offset, long length, - List locations, - Map storageDirIds) + List locations) { this(); this.blockId = blockId; @@ -166,7 +156,6 @@ public ClientBlockInfo( this.length = length; setLengthIsSet(true); this.locations = locations; - this.storageDirIds = storageDirIds; } /** @@ -184,21 +173,6 @@ public ClientBlockInfo(ClientBlockInfo other) { } this.locations = __this__locations; } - if (other.isSetStorageDirIds()) { - Map __this__storageDirIds = new HashMap(other.storageDirIds.size()); - for (Map.Entry other_element : other.storageDirIds.entrySet()) { - - NetAddress other_element_key = other_element.getKey(); - Long other_element_value = other_element.getValue(); - - NetAddress __this__storageDirIds_copy_key = new NetAddress(other_element_key); - - Long __this__storageDirIds_copy_value = other_element_value; - - __this__storageDirIds.put(__this__storageDirIds_copy_key, __this__storageDirIds_copy_value); - } - this.storageDirIds = __this__storageDirIds; - } } public ClientBlockInfo deepCopy() { @@ -214,7 +188,6 @@ public void clear() { setLengthIsSet(false); this.length = 0; this.locations = null; - this.storageDirIds = null; } public long getBlockId() { @@ -325,41 +298,6 @@ public void setLocationsIsSet(boolean value) { } } - public int getStorageDirIdsSize() { - return (this.storageDirIds == null) ? 0 : this.storageDirIds.size(); - } - - public void putToStorageDirIds(NetAddress key, long val) { - if (this.storageDirIds == null) { - this.storageDirIds = new HashMap(); - } - this.storageDirIds.put(key, val); - } - - public Map getStorageDirIds() { - return this.storageDirIds; - } - - public ClientBlockInfo setStorageDirIds(Map storageDirIds) { - this.storageDirIds = storageDirIds; - return this; - } - - public void unsetStorageDirIds() { - this.storageDirIds = null; - } - - /** Returns true if field storageDirIds is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirIds() { - return this.storageDirIds != null; - } - - public void setStorageDirIdsIsSet(boolean value) { - if (!value) { - this.storageDirIds = null; - } - } - public void setFieldValue(_Fields field, Object value) { switch (field) { case BLOCK_ID: @@ -394,14 +332,6 @@ public void setFieldValue(_Fields field, Object value) { } break; - case STORAGE_DIR_IDS: - if (value == null) { - unsetStorageDirIds(); - } else { - setStorageDirIds((Map)value); - } - break; - } } @@ -419,9 +349,6 @@ public Object getFieldValue(_Fields field) { case LOCATIONS: return getLocations(); - case STORAGE_DIR_IDS: - return getStorageDirIds(); - } throw new IllegalStateException(); } @@ -441,8 +368,6 @@ public boolean isSet(_Fields field) { return isSetLength(); case LOCATIONS: return isSetLocations(); - case STORAGE_DIR_IDS: - return isSetStorageDirIds(); } throw new IllegalStateException(); } @@ -496,15 +421,6 @@ public boolean equals(ClientBlockInfo that) { return false; } - boolean this_present_storageDirIds = true && this.isSetStorageDirIds(); - boolean that_present_storageDirIds = true && that.isSetStorageDirIds(); - if (this_present_storageDirIds || that_present_storageDirIds) { - if (!(this_present_storageDirIds && that_present_storageDirIds)) - return false; - if (!this.storageDirIds.equals(that.storageDirIds)) - return false; - } - return true; } @@ -561,16 +477,6 @@ public int compareTo(ClientBlockInfo other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStorageDirIds()).compareTo(other.isSetStorageDirIds()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirIds()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirIds, other.storageDirIds); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -610,14 +516,6 @@ public String toString() { sb.append(this.locations); } first = false; - if (!first) sb.append(", "); - sb.append("storageDirIds:"); - if (this.storageDirIds == null) { - sb.append("null"); - } else { - sb.append(this.storageDirIds); - } - first = false; sb.append(")"); return sb.toString(); } @@ -706,27 +604,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientBlockInfo str org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 5: // STORAGE_DIR_IDS - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map3 = iprot.readMapBegin(); - struct.storageDirIds = new HashMap(2*_map3.size); - for (int _i4 = 0; _i4 < _map3.size; ++_i4) - { - NetAddress _key5; - long _val6; - _key5 = new NetAddress(); - _key5.read(iprot); - _val6 = iprot.readI64(); - struct.storageDirIds.put(_key5, _val6); - } - iprot.readMapEnd(); - } - struct.setStorageDirIdsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -755,27 +632,14 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientBlockInfo st oprot.writeFieldBegin(LOCATIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locations.size())); - for (NetAddress _iter7 : struct.locations) + for (NetAddress _iter3 : struct.locations) { - _iter7.write(oprot); + _iter3.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } - if (struct.storageDirIds != null) { - oprot.writeFieldBegin(STORAGE_DIR_IDS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, struct.storageDirIds.size())); - for (Map.Entry _iter8 : struct.storageDirIds.entrySet()) - { - _iter8.getKey().write(oprot); - oprot.writeI64(_iter8.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -806,10 +670,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientBlockInfo str if (struct.isSetLocations()) { optionals.set(3); } - if (struct.isSetStorageDirIds()) { - optionals.set(4); - } - oprot.writeBitSet(optionals, 5); + oprot.writeBitSet(optionals, 4); if (struct.isSetBlockId()) { oprot.writeI64(struct.blockId); } @@ -822,19 +683,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientBlockInfo str if (struct.isSetLocations()) { { oprot.writeI32(struct.locations.size()); - for (NetAddress _iter9 : struct.locations) + for (NetAddress _iter4 : struct.locations) { - _iter9.write(oprot); - } - } - } - if (struct.isSetStorageDirIds()) { - { - oprot.writeI32(struct.storageDirIds.size()); - for (Map.Entry _iter10 : struct.storageDirIds.entrySet()) - { - _iter10.getKey().write(oprot); - oprot.writeI64(_iter10.getValue()); + _iter4.write(oprot); } } } @@ -843,7 +694,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientBlockInfo str @Override public void read(org.apache.thrift.protocol.TProtocol prot, ClientBlockInfo struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(5); + BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); @@ -858,34 +709,18 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ClientBlockInfo stru } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list11 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.locations = new ArrayList(_list11.size); - for (int _i12 = 0; _i12 < _list11.size; ++_i12) + org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.locations = new ArrayList(_list5.size); + for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - NetAddress _elem13; - _elem13 = new NetAddress(); - _elem13.read(iprot); - struct.locations.add(_elem13); + NetAddress _elem7; + _elem7 = new NetAddress(); + _elem7.read(iprot); + struct.locations.add(_elem7); } } struct.setLocationsIsSet(true); } - if (incoming.get(4)) { - { - org.apache.thrift.protocol.TMap _map14 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.storageDirIds = new HashMap(2*_map14.size); - for (int _i15 = 0; _i15 < _map14.size; ++_i15) - { - NetAddress _key16; - long _val17; - _key16 = new NetAddress(); - _key16.read(iprot); - _val17 = iprot.readI64(); - struct.storageDirIds.put(_key16, _val17); - } - } - struct.setStorageDirIdsIsSet(true); - } } } diff --git a/core/src/main/java/tachyon/thrift/ClientDependencyInfo.java b/core/src/main/java/tachyon/thrift/ClientDependencyInfo.java index 6de29fa49ba3..6e9831edd635 100644 --- a/core/src/main/java/tachyon/thrift/ClientDependencyInfo.java +++ b/core/src/main/java/tachyon/thrift/ClientDependencyInfo.java @@ -611,13 +611,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientDependencyInf case 2: // PARENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list26 = iprot.readListBegin(); - struct.parents = new ArrayList(_list26.size); - for (int _i27 = 0; _i27 < _list26.size; ++_i27) + org.apache.thrift.protocol.TList _list16 = iprot.readListBegin(); + struct.parents = new ArrayList(_list16.size); + for (int _i17 = 0; _i17 < _list16.size; ++_i17) { - int _elem28; - _elem28 = iprot.readI32(); - struct.parents.add(_elem28); + int _elem18; + _elem18 = iprot.readI32(); + struct.parents.add(_elem18); } iprot.readListEnd(); } @@ -629,13 +629,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientDependencyInf case 3: // CHILDREN if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list29 = iprot.readListBegin(); - struct.children = new ArrayList(_list29.size); - for (int _i30 = 0; _i30 < _list29.size; ++_i30) + org.apache.thrift.protocol.TList _list19 = iprot.readListBegin(); + struct.children = new ArrayList(_list19.size); + for (int _i20 = 0; _i20 < _list19.size; ++_i20) { - int _elem31; - _elem31 = iprot.readI32(); - struct.children.add(_elem31); + int _elem21; + _elem21 = iprot.readI32(); + struct.children.add(_elem21); } iprot.readListEnd(); } @@ -647,13 +647,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientDependencyInf case 4: // DATA if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list32 = iprot.readListBegin(); - struct.data = new ArrayList(_list32.size); - for (int _i33 = 0; _i33 < _list32.size; ++_i33) + org.apache.thrift.protocol.TList _list22 = iprot.readListBegin(); + struct.data = new ArrayList(_list22.size); + for (int _i23 = 0; _i23 < _list22.size; ++_i23) { - ByteBuffer _elem34; - _elem34 = iprot.readBinary(); - struct.data.add(_elem34); + ByteBuffer _elem24; + _elem24 = iprot.readBinary(); + struct.data.add(_elem24); } iprot.readListEnd(); } @@ -684,9 +684,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientDependencyIn oprot.writeFieldBegin(PARENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.parents.size())); - for (int _iter35 : struct.parents) + for (int _iter25 : struct.parents) { - oprot.writeI32(_iter35); + oprot.writeI32(_iter25); } oprot.writeListEnd(); } @@ -696,9 +696,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientDependencyIn oprot.writeFieldBegin(CHILDREN_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.children.size())); - for (int _iter36 : struct.children) + for (int _iter26 : struct.children) { - oprot.writeI32(_iter36); + oprot.writeI32(_iter26); } oprot.writeListEnd(); } @@ -708,9 +708,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientDependencyIn oprot.writeFieldBegin(DATA_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.data.size())); - for (ByteBuffer _iter37 : struct.data) + for (ByteBuffer _iter27 : struct.data) { - oprot.writeBinary(_iter37); + oprot.writeBinary(_iter27); } oprot.writeListEnd(); } @@ -753,27 +753,27 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientDependencyInf if (struct.isSetParents()) { { oprot.writeI32(struct.parents.size()); - for (int _iter38 : struct.parents) + for (int _iter28 : struct.parents) { - oprot.writeI32(_iter38); + oprot.writeI32(_iter28); } } } if (struct.isSetChildren()) { { oprot.writeI32(struct.children.size()); - for (int _iter39 : struct.children) + for (int _iter29 : struct.children) { - oprot.writeI32(_iter39); + oprot.writeI32(_iter29); } } } if (struct.isSetData()) { { oprot.writeI32(struct.data.size()); - for (ByteBuffer _iter40 : struct.data) + for (ByteBuffer _iter30 : struct.data) { - oprot.writeBinary(_iter40); + oprot.writeBinary(_iter30); } } } @@ -789,39 +789,39 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ClientDependencyInfo } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list41 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.parents = new ArrayList(_list41.size); - for (int _i42 = 0; _i42 < _list41.size; ++_i42) + org.apache.thrift.protocol.TList _list31 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.parents = new ArrayList(_list31.size); + for (int _i32 = 0; _i32 < _list31.size; ++_i32) { - int _elem43; - _elem43 = iprot.readI32(); - struct.parents.add(_elem43); + int _elem33; + _elem33 = iprot.readI32(); + struct.parents.add(_elem33); } } struct.setParentsIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list44 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.children = new ArrayList(_list44.size); - for (int _i45 = 0; _i45 < _list44.size; ++_i45) + org.apache.thrift.protocol.TList _list34 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.children = new ArrayList(_list34.size); + for (int _i35 = 0; _i35 < _list34.size; ++_i35) { - int _elem46; - _elem46 = iprot.readI32(); - struct.children.add(_elem46); + int _elem36; + _elem36 = iprot.readI32(); + struct.children.add(_elem36); } } struct.setChildrenIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list47 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.data = new ArrayList(_list47.size); - for (int _i48 = 0; _i48 < _list47.size; ++_i48) + org.apache.thrift.protocol.TList _list37 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.data = new ArrayList(_list37.size); + for (int _i38 = 0; _i38 < _list37.size; ++_i38) { - ByteBuffer _elem49; - _elem49 = iprot.readBinary(); - struct.data.add(_elem49); + ByteBuffer _elem39; + _elem39 = iprot.readBinary(); + struct.data.add(_elem39); } } struct.setDataIsSet(true); diff --git a/core/src/main/java/tachyon/thrift/ClientFileInfo.java b/core/src/main/java/tachyon/thrift/ClientFileInfo.java index 33f115286b37..62518e8a684b 100644 --- a/core/src/main/java/tachyon/thrift/ClientFileInfo.java +++ b/core/src/main/java/tachyon/thrift/ClientFileInfo.java @@ -1464,13 +1464,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClientFileInfo stru case 12: // BLOCK_IDS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list18 = iprot.readListBegin(); - struct.blockIds = new ArrayList(_list18.size); - for (int _i19 = 0; _i19 < _list18.size; ++_i19) + org.apache.thrift.protocol.TList _list8 = iprot.readListBegin(); + struct.blockIds = new ArrayList(_list8.size); + for (int _i9 = 0; _i9 < _list8.size; ++_i9) { - long _elem20; - _elem20 = iprot.readI64(); - struct.blockIds.add(_elem20); + long _elem10; + _elem10 = iprot.readI64(); + struct.blockIds.add(_elem10); } iprot.readListEnd(); } @@ -1561,9 +1561,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClientFileInfo str oprot.writeFieldBegin(BLOCK_IDS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.blockIds.size())); - for (long _iter21 : struct.blockIds) + for (long _iter11 : struct.blockIds) { - oprot.writeI64(_iter21); + oprot.writeI64(_iter11); } oprot.writeListEnd(); } @@ -1678,9 +1678,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClientFileInfo stru if (struct.isSetBlockIds()) { { oprot.writeI32(struct.blockIds.size()); - for (long _iter22 : struct.blockIds) + for (long _iter12 : struct.blockIds) { - oprot.writeI64(_iter22); + oprot.writeI64(_iter12); } } } @@ -1745,13 +1745,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ClientFileInfo struc } if (incoming.get(11)) { { - org.apache.thrift.protocol.TList _list23 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.blockIds = new ArrayList(_list23.size); - for (int _i24 = 0; _i24 < _list23.size; ++_i24) + org.apache.thrift.protocol.TList _list13 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.blockIds = new ArrayList(_list13.size); + for (int _i14 = 0; _i14 < _list13.size; ++_i14) { - long _elem25; - _elem25 = iprot.readI64(); - struct.blockIds.add(_elem25); + long _elem15; + _elem15 = iprot.readI64(); + struct.blockIds.add(_elem15); } } struct.setBlockIdsIsSet(true); diff --git a/core/src/main/java/tachyon/thrift/ClientLocationInfo.java b/core/src/main/java/tachyon/thrift/ClientLocationInfo.java deleted file mode 100644 index d9f77d42cbd0..000000000000 --- a/core/src/main/java/tachyon/thrift/ClientLocationInfo.java +++ /dev/null @@ -1,486 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.9.1) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -package tachyon.thrift; - -import org.apache.thrift.scheme.IScheme; -import org.apache.thrift.scheme.SchemeFactory; -import org.apache.thrift.scheme.StandardScheme; - -import org.apache.thrift.scheme.TupleScheme; -import org.apache.thrift.protocol.TTupleProtocol; -import org.apache.thrift.protocol.TProtocolException; -import org.apache.thrift.EncodingUtils; -import org.apache.thrift.TException; -import org.apache.thrift.async.AsyncMethodCallback; -import org.apache.thrift.server.AbstractNonblockingServer.*; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ClientLocationInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClientLocationInfo"); - - private static final org.apache.thrift.protocol.TField STORAGE_DIR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storageDirId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("path", org.apache.thrift.protocol.TType.STRING, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new ClientLocationInfoStandardSchemeFactory()); - schemes.put(TupleScheme.class, new ClientLocationInfoTupleSchemeFactory()); - } - - public long storageDirId; // required - public String path; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - STORAGE_DIR_ID((short)1, "storageDirId"), - PATH((short)2, "path"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 2: // PATH - return PATH; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __STORAGEDIRID_ISSET_ID = 0; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.PATH, new org.apache.thrift.meta_data.FieldMetaData("path", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClientLocationInfo.class, metaDataMap); - } - - public ClientLocationInfo() { - } - - public ClientLocationInfo( - long storageDirId, - String path) - { - this(); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - this.path = path; - } - - /** - * Performs a deep copy on other. - */ - public ClientLocationInfo(ClientLocationInfo other) { - __isset_bitfield = other.__isset_bitfield; - this.storageDirId = other.storageDirId; - if (other.isSetPath()) { - this.path = other.path; - } - } - - public ClientLocationInfo deepCopy() { - return new ClientLocationInfo(this); - } - - @Override - public void clear() { - setStorageDirIdIsSet(false); - this.storageDirId = 0; - this.path = null; - } - - public long getStorageDirId() { - return this.storageDirId; - } - - public ClientLocationInfo setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - - public String getPath() { - return this.path; - } - - public ClientLocationInfo setPath(String path) { - this.path = path; - return this; - } - - public void unsetPath() { - this.path = null; - } - - /** Returns true if field path is set (has been assigned a value) and false otherwise */ - public boolean isSetPath() { - return this.path != null; - } - - public void setPathIsSet(boolean value) { - if (!value) { - this.path = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - - case PATH: - if (value == null) { - unsetPath(); - } else { - setPath((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - - case PATH: - return getPath(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STORAGE_DIR_ID: - return isSetStorageDirId(); - case PATH: - return isSetPath(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ClientLocationInfo) - return this.equals((ClientLocationInfo)that); - return false; - } - - public boolean equals(ClientLocationInfo that) { - if (that == null) - return false; - - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - - boolean this_present_path = true && this.isSetPath(); - boolean that_present_path = true && that.isSetPath(); - if (this_present_path || that_present_path) { - if (!(this_present_path && that_present_path)) - return false; - if (!this.path.equals(that.path)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public int compareTo(ClientLocationInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetPath()).compareTo(other.isSetPath()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetPath()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.path, other.path); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ClientLocationInfo("); - boolean first = true; - - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); - sb.append("path:"); - if (this.path == null) { - sb.append("null"); - } else { - sb.append(this.path); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class ClientLocationInfoStandardSchemeFactory implements SchemeFactory { - public ClientLocationInfoStandardScheme getScheme() { - return new ClientLocationInfoStandardScheme(); - } - } - - private static class ClientLocationInfoStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, ClientLocationInfo struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // PATH - if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { - struct.path = iprot.readString(); - struct.setPathIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - // check for required fields of primitive type, which can't be checked in the validate method - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, ClientLocationInfo struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); - if (struct.path != null) { - oprot.writeFieldBegin(PATH_FIELD_DESC); - oprot.writeString(struct.path); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class ClientLocationInfoTupleSchemeFactory implements SchemeFactory { - public ClientLocationInfoTupleScheme getScheme() { - return new ClientLocationInfoTupleScheme(); - } - } - - private static class ClientLocationInfoTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, ClientLocationInfo struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetStorageDirId()) { - optionals.set(0); - } - if (struct.isSetPath()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } - if (struct.isSetPath()) { - oprot.writeString(struct.path); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, ClientLocationInfo struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } - if (incoming.get(1)) { - struct.path = iprot.readString(); - struct.setPathIsSet(true); - } - } - } - -} - diff --git a/core/src/main/java/tachyon/thrift/Command.java b/core/src/main/java/tachyon/thrift/Command.java index d7369451e282..9aa59e81b78d 100644 --- a/core/src/main/java/tachyon/thrift/Command.java +++ b/core/src/main/java/tachyon/thrift/Command.java @@ -439,13 +439,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Command struct) thr case 2: // M_DATA if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list50 = iprot.readListBegin(); - struct.mData = new ArrayList(_list50.size); - for (int _i51 = 0; _i51 < _list50.size; ++_i51) + org.apache.thrift.protocol.TList _list40 = iprot.readListBegin(); + struct.mData = new ArrayList(_list40.size); + for (int _i41 = 0; _i41 < _list40.size; ++_i41) { - long _elem52; - _elem52 = iprot.readI64(); - struct.mData.add(_elem52); + long _elem42; + _elem42 = iprot.readI64(); + struct.mData.add(_elem42); } iprot.readListEnd(); } @@ -478,9 +478,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Command struct) th oprot.writeFieldBegin(M_DATA_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.mData.size())); - for (long _iter53 : struct.mData) + for (long _iter43 : struct.mData) { - oprot.writeI64(_iter53); + oprot.writeI64(_iter43); } oprot.writeListEnd(); } @@ -517,9 +517,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Command struct) thr if (struct.isSetMData()) { { oprot.writeI32(struct.mData.size()); - for (long _iter54 : struct.mData) + for (long _iter44 : struct.mData) { - oprot.writeI64(_iter54); + oprot.writeI64(_iter44); } } } @@ -535,13 +535,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Command struct) thro } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list55 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.mData = new ArrayList(_list55.size); - for (int _i56 = 0; _i56 < _list55.size; ++_i56) + org.apache.thrift.protocol.TList _list45 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.mData = new ArrayList(_list45.size); + for (int _i46 = 0; _i46 < _list45.size; ++_i46) { - long _elem57; - _elem57 = iprot.readI64(); - struct.mData.add(_elem57); + long _elem47; + _elem47 = iprot.readI64(); + struct.mData.add(_elem47); } } struct.setMDataIsSet(true); diff --git a/core/src/main/java/tachyon/thrift/MasterService.java b/core/src/main/java/tachyon/thrift/MasterService.java index d8455dcef64e..caf0f31989ce 100644 --- a/core/src/main/java/tachyon/thrift/MasterService.java +++ b/core/src/main/java/tachyon/thrift/MasterService.java @@ -51,11 +51,11 @@ public interface Iface { * @param usedBytes * @param currentBlocks */ - public long worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks) throws BlockInfoException, org.apache.thrift.TException; + public long worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks) throws BlockInfoException, org.apache.thrift.TException; - public Command worker_heartbeat(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds) throws BlockInfoException, org.apache.thrift.TException; + public Command worker_heartbeat(long workerId, long usedBytes, List removedBlocks) throws BlockInfoException, org.apache.thrift.TException; - public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException; + public void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException; public Set worker_getPinIdList() throws org.apache.thrift.TException; @@ -152,11 +152,11 @@ public interface AsyncIface { public void liststatus(String path, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void worker_heartbeat(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void worker_heartbeat(long workerId, long usedBytes, List removedBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void worker_getPinIdList(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -318,13 +318,13 @@ public List recv_liststatus() throws InvalidPathException, FileD throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "liststatus failed: unknown result"); } - public long worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks) throws BlockInfoException, org.apache.thrift.TException + public long worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks) throws BlockInfoException, org.apache.thrift.TException { send_worker_register(workerNetAddress, totalBytes, usedBytes, currentBlocks); return recv_worker_register(); } - public void send_worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks) throws org.apache.thrift.TException + public void send_worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks) throws org.apache.thrift.TException { worker_register_args args = new worker_register_args(); args.setWorkerNetAddress(workerNetAddress); @@ -347,19 +347,18 @@ public long recv_worker_register() throws BlockInfoException, org.apache.thrift. throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "worker_register failed: unknown result"); } - public Command worker_heartbeat(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds) throws BlockInfoException, org.apache.thrift.TException + public Command worker_heartbeat(long workerId, long usedBytes, List removedBlocks) throws BlockInfoException, org.apache.thrift.TException { - send_worker_heartbeat(workerId, usedBytes, removedBlockIds, evictedBlockIds); + send_worker_heartbeat(workerId, usedBytes, removedBlocks); return recv_worker_heartbeat(); } - public void send_worker_heartbeat(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds) throws org.apache.thrift.TException + public void send_worker_heartbeat(long workerId, long usedBytes, List removedBlocks) throws org.apache.thrift.TException { worker_heartbeat_args args = new worker_heartbeat_args(); args.setWorkerId(workerId); args.setUsedBytes(usedBytes); - args.setRemovedBlockIds(removedBlockIds); - args.setEvictedBlockIds(evictedBlockIds); + args.setRemovedBlocks(removedBlocks); sendBase("worker_heartbeat", args); } @@ -376,18 +375,17 @@ public Command recv_worker_heartbeat() throws BlockInfoException, org.apache.thr throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "worker_heartbeat failed: unknown result"); } - public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException + public void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException { - send_worker_cacheBlock(workerId, workerUsedBytes, storageDirId, blockId, length); + send_worker_cacheBlock(workerId, workerUsedBytes, blockId, length); recv_worker_cacheBlock(); } - public void send_worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length) throws org.apache.thrift.TException + public void send_worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length) throws org.apache.thrift.TException { worker_cacheBlock_args args = new worker_cacheBlock_args(); args.setWorkerId(workerId); args.setWorkerUsedBytes(workerUsedBytes); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); args.setLength(length); sendBase("worker_cacheBlock", args); @@ -1251,7 +1249,7 @@ public List getResult() throws InvalidPathException, FileDoesNot } } - public void worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void worker_register(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); worker_register_call method_call = new worker_register_call(workerNetAddress, totalBytes, usedBytes, currentBlocks, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1262,8 +1260,8 @@ public static class worker_register_call extends org.apache.thrift.async.TAsyncM private NetAddress workerNetAddress; private long totalBytes; private long usedBytes; - private Map> currentBlocks; - public worker_register_call(NetAddress workerNetAddress, long totalBytes, long usedBytes, Map> currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + private List currentBlocks; + public worker_register_call(NetAddress workerNetAddress, long totalBytes, long usedBytes, List currentBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.workerNetAddress = workerNetAddress; this.totalBytes = totalBytes; @@ -1292,9 +1290,9 @@ public long getResult() throws BlockInfoException, org.apache.thrift.TException } } - public void worker_heartbeat(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void worker_heartbeat(long workerId, long usedBytes, List removedBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - worker_heartbeat_call method_call = new worker_heartbeat_call(workerId, usedBytes, removedBlockIds, evictedBlockIds, resultHandler, this, ___protocolFactory, ___transport); + worker_heartbeat_call method_call = new worker_heartbeat_call(workerId, usedBytes, removedBlocks, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } @@ -1302,14 +1300,12 @@ public void worker_heartbeat(long workerId, long usedBytes, List removedBl public static class worker_heartbeat_call extends org.apache.thrift.async.TAsyncMethodCall { private long workerId; private long usedBytes; - private List removedBlockIds; - private Map> evictedBlockIds; - public worker_heartbeat_call(long workerId, long usedBytes, List removedBlockIds, Map> evictedBlockIds, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + private List removedBlocks; + public worker_heartbeat_call(long workerId, long usedBytes, List removedBlocks, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.workerId = workerId; this.usedBytes = usedBytes; - this.removedBlockIds = removedBlockIds; - this.evictedBlockIds = evictedBlockIds; + this.removedBlocks = removedBlocks; } public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { @@ -1317,8 +1313,7 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa worker_heartbeat_args args = new worker_heartbeat_args(); args.setWorkerId(workerId); args.setUsedBytes(usedBytes); - args.setRemovedBlockIds(removedBlockIds); - args.setEvictedBlockIds(evictedBlockIds); + args.setRemovedBlocks(removedBlocks); args.write(prot); prot.writeMessageEnd(); } @@ -1333,9 +1328,9 @@ public Command getResult() throws BlockInfoException, org.apache.thrift.TExcepti } } - public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void worker_cacheBlock(long workerId, long workerUsedBytes, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - worker_cacheBlock_call method_call = new worker_cacheBlock_call(workerId, workerUsedBytes, storageDirId, blockId, length, resultHandler, this, ___protocolFactory, ___transport); + worker_cacheBlock_call method_call = new worker_cacheBlock_call(workerId, workerUsedBytes, blockId, length, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } @@ -1343,14 +1338,12 @@ public void worker_cacheBlock(long workerId, long workerUsedBytes, long storageD public static class worker_cacheBlock_call extends org.apache.thrift.async.TAsyncMethodCall { private long workerId; private long workerUsedBytes; - private long storageDirId; private long blockId; private long length; - public worker_cacheBlock_call(long workerId, long workerUsedBytes, long storageDirId, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public worker_cacheBlock_call(long workerId, long workerUsedBytes, long blockId, long length, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.workerId = workerId; this.workerUsedBytes = workerUsedBytes; - this.storageDirId = storageDirId; this.blockId = blockId; this.length = length; } @@ -1360,7 +1353,6 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa worker_cacheBlock_args args = new worker_cacheBlock_args(); args.setWorkerId(workerId); args.setWorkerUsedBytes(workerUsedBytes); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); args.setLength(length); args.write(prot); @@ -2439,7 +2431,7 @@ protected boolean isOneway() { public worker_heartbeat_result getResult(I iface, worker_heartbeat_args args) throws org.apache.thrift.TException { worker_heartbeat_result result = new worker_heartbeat_result(); try { - result.success = iface.worker_heartbeat(args.workerId, args.usedBytes, args.removedBlockIds, args.evictedBlockIds); + result.success = iface.worker_heartbeat(args.workerId, args.usedBytes, args.removedBlocks); } catch (BlockInfoException e) { result.e = e; } @@ -2463,7 +2455,7 @@ protected boolean isOneway() { public worker_cacheBlock_result getResult(I iface, worker_cacheBlock_args args) throws org.apache.thrift.TException { worker_cacheBlock_result result = new worker_cacheBlock_result(); try { - iface.worker_cacheBlock(args.workerId, args.workerUsedBytes, args.storageDirId, args.blockId, args.length); + iface.worker_cacheBlock(args.workerId, args.workerUsedBytes, args.blockId, args.length); } catch (FileDoesNotExistException eP) { result.eP = eP; } catch (SuspectedFileSizeException eS) { @@ -3468,7 +3460,7 @@ protected boolean isOneway() { } public void start(I iface, worker_heartbeat_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.worker_heartbeat(args.workerId, args.usedBytes, args.removedBlockIds, args.evictedBlockIds,resultHandler); + iface.worker_heartbeat(args.workerId, args.usedBytes, args.removedBlocks,resultHandler); } } @@ -3534,7 +3526,7 @@ protected boolean isOneway() { } public void start(I iface, worker_cacheBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.worker_cacheBlock(args.workerId, args.workerUsedBytes, args.storageDirId, args.blockId, args.length,resultHandler); + iface.worker_cacheBlock(args.workerId, args.workerUsedBytes, args.blockId, args.length,resultHandler); } } @@ -6947,14 +6939,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getWorkersInfo_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list58 = iprot.readListBegin(); - struct.success = new ArrayList(_list58.size); - for (int _i59 = 0; _i59 < _list58.size; ++_i59) + org.apache.thrift.protocol.TList _list48 = iprot.readListBegin(); + struct.success = new ArrayList(_list48.size); + for (int _i49 = 0; _i49 < _list48.size; ++_i49) { - ClientWorkerInfo _elem60; - _elem60 = new ClientWorkerInfo(); - _elem60.read(iprot); - struct.success.add(_elem60); + ClientWorkerInfo _elem50; + _elem50 = new ClientWorkerInfo(); + _elem50.read(iprot); + struct.success.add(_elem50); } iprot.readListEnd(); } @@ -6982,9 +6974,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getWorkersInfo_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ClientWorkerInfo _iter61 : struct.success) + for (ClientWorkerInfo _iter51 : struct.success) { - _iter61.write(oprot); + _iter51.write(oprot); } oprot.writeListEnd(); } @@ -7015,9 +7007,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getWorkersInfo_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ClientWorkerInfo _iter62 : struct.success) + for (ClientWorkerInfo _iter52 : struct.success) { - _iter62.write(oprot); + _iter52.write(oprot); } } } @@ -7029,14 +7021,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getWorkersInfo_resul BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list63 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list63.size); - for (int _i64 = 0; _i64 < _list63.size; ++_i64) + org.apache.thrift.protocol.TList _list53 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list53.size); + for (int _i54 = 0; _i54 < _list53.size; ++_i54) { - ClientWorkerInfo _elem65; - _elem65 = new ClientWorkerInfo(); - _elem65.read(iprot); - struct.success.add(_elem65); + ClientWorkerInfo _elem55; + _elem55 = new ClientWorkerInfo(); + _elem55.read(iprot); + struct.success.add(_elem55); } } struct.setSuccessIsSet(true); @@ -7863,14 +7855,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, liststatus_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list66 = iprot.readListBegin(); - struct.success = new ArrayList(_list66.size); - for (int _i67 = 0; _i67 < _list66.size; ++_i67) + org.apache.thrift.protocol.TList _list56 = iprot.readListBegin(); + struct.success = new ArrayList(_list56.size); + for (int _i57 = 0; _i57 < _list56.size; ++_i57) { - ClientFileInfo _elem68; - _elem68 = new ClientFileInfo(); - _elem68.read(iprot); - struct.success.add(_elem68); + ClientFileInfo _elem58; + _elem58 = new ClientFileInfo(); + _elem58.read(iprot); + struct.success.add(_elem58); } iprot.readListEnd(); } @@ -7916,9 +7908,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, liststatus_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ClientFileInfo _iter69 : struct.success) + for (ClientFileInfo _iter59 : struct.success) { - _iter69.write(oprot); + _iter59.write(oprot); } oprot.writeListEnd(); } @@ -7965,9 +7957,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, liststatus_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ClientFileInfo _iter70 : struct.success) + for (ClientFileInfo _iter60 : struct.success) { - _iter70.write(oprot); + _iter60.write(oprot); } } } @@ -7985,14 +7977,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, liststatus_result st BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list71 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list71.size); - for (int _i72 = 0; _i72 < _list71.size; ++_i72) + org.apache.thrift.protocol.TList _list61 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list61.size); + for (int _i62 = 0; _i62 < _list61.size; ++_i62) { - ClientFileInfo _elem73; - _elem73 = new ClientFileInfo(); - _elem73.read(iprot); - struct.success.add(_elem73); + ClientFileInfo _elem63; + _elem63 = new ClientFileInfo(); + _elem63.read(iprot); + struct.success.add(_elem63); } } struct.setSuccessIsSet(true); @@ -8018,7 +8010,7 @@ public static class worker_register_args implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -8029,7 +8021,7 @@ public static class worker_register_args implements org.apache.thrift.TBase> currentBlocks; // required + public List currentBlocks; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { @@ -8112,10 +8104,8 @@ public String getFieldName() { tmpMap.put(_Fields.USED_BYTES, new org.apache.thrift.meta_data.FieldMetaData("usedBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.CURRENT_BLOCKS, new org.apache.thrift.meta_data.FieldMetaData("currentBlocks", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64), - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))))); + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(worker_register_args.class, metaDataMap); } @@ -8127,7 +8117,7 @@ public worker_register_args( NetAddress workerNetAddress, long totalBytes, long usedBytes, - Map> currentBlocks) + List currentBlocks) { this(); this.workerNetAddress = workerNetAddress; @@ -8149,18 +8139,7 @@ public worker_register_args(worker_register_args other) { this.totalBytes = other.totalBytes; this.usedBytes = other.usedBytes; if (other.isSetCurrentBlocks()) { - Map> __this__currentBlocks = new HashMap>(other.currentBlocks.size()); - for (Map.Entry> other_element : other.currentBlocks.entrySet()) { - - Long other_element_key = other_element.getKey(); - List other_element_value = other_element.getValue(); - - Long __this__currentBlocks_copy_key = other_element_key; - - List __this__currentBlocks_copy_value = new ArrayList(other_element_value); - - __this__currentBlocks.put(__this__currentBlocks_copy_key, __this__currentBlocks_copy_value); - } + List __this__currentBlocks = new ArrayList(other.currentBlocks); this.currentBlocks = __this__currentBlocks; } } @@ -8253,18 +8232,22 @@ public int getCurrentBlocksSize() { return (this.currentBlocks == null) ? 0 : this.currentBlocks.size(); } - public void putToCurrentBlocks(long key, List val) { + public java.util.Iterator getCurrentBlocksIterator() { + return (this.currentBlocks == null) ? null : this.currentBlocks.iterator(); + } + + public void addToCurrentBlocks(long elem) { if (this.currentBlocks == null) { - this.currentBlocks = new HashMap>(); + this.currentBlocks = new ArrayList(); } - this.currentBlocks.put(key, val); + this.currentBlocks.add(elem); } - public Map> getCurrentBlocks() { + public List getCurrentBlocks() { return this.currentBlocks; } - public worker_register_args setCurrentBlocks(Map> currentBlocks) { + public worker_register_args setCurrentBlocks(List currentBlocks) { this.currentBlocks = currentBlocks; return this; } @@ -8314,7 +8297,7 @@ public void setFieldValue(_Fields field, Object value) { if (value == null) { unsetCurrentBlocks(); } else { - setCurrentBlocks((Map>)value); + setCurrentBlocks((List)value); } break; @@ -8580,29 +8563,17 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_register_arg } break; case 4: // CURRENT_BLOCKS - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(); - struct.currentBlocks = new HashMap>(2*_map74.size); - for (int _i75 = 0; _i75 < _map74.size; ++_i75) + org.apache.thrift.protocol.TList _list64 = iprot.readListBegin(); + struct.currentBlocks = new ArrayList(_list64.size); + for (int _i65 = 0; _i65 < _list64.size; ++_i65) { - long _key76; - List _val77; - _key76 = iprot.readI64(); - { - org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); - _val77 = new ArrayList(_list78.size); - for (int _i79 = 0; _i79 < _list78.size; ++_i79) - { - long _elem80; - _elem80 = iprot.readI64(); - _val77.add(_elem80); - } - iprot.readListEnd(); - } - struct.currentBlocks.put(_key76, _val77); + long _elem66; + _elem66 = iprot.readI64(); + struct.currentBlocks.add(_elem66); } - iprot.readMapEnd(); + iprot.readListEnd(); } struct.setCurrentBlocksIsSet(true); } else { @@ -8638,20 +8609,12 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, worker_register_ar if (struct.currentBlocks != null) { oprot.writeFieldBegin(CURRENT_BLOCKS_FIELD_DESC); { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.LIST, struct.currentBlocks.size())); - for (Map.Entry> _iter81 : struct.currentBlocks.entrySet()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.currentBlocks.size())); + for (long _iter67 : struct.currentBlocks) { - oprot.writeI64(_iter81.getKey()); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter81.getValue().size())); - for (long _iter82 : _iter81.getValue()) - { - oprot.writeI64(_iter82); - } - oprot.writeListEnd(); - } + oprot.writeI64(_iter67); } - oprot.writeMapEnd(); + oprot.writeListEnd(); } oprot.writeFieldEnd(); } @@ -8698,16 +8661,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_register_arg if (struct.isSetCurrentBlocks()) { { oprot.writeI32(struct.currentBlocks.size()); - for (Map.Entry> _iter83 : struct.currentBlocks.entrySet()) + for (long _iter68 : struct.currentBlocks) { - oprot.writeI64(_iter83.getKey()); - { - oprot.writeI32(_iter83.getValue().size()); - for (long _iter84 : _iter83.getValue()) - { - oprot.writeI64(_iter84); - } - } + oprot.writeI64(_iter68); } } } @@ -8732,24 +8688,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, worker_register_args } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map85 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.LIST, iprot.readI32()); - struct.currentBlocks = new HashMap>(2*_map85.size); - for (int _i86 = 0; _i86 < _map85.size; ++_i86) + org.apache.thrift.protocol.TList _list69 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.currentBlocks = new ArrayList(_list69.size); + for (int _i70 = 0; _i70 < _list69.size; ++_i70) { - long _key87; - List _val88; - _key87 = iprot.readI64(); - { - org.apache.thrift.protocol.TList _list89 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _val88 = new ArrayList(_list89.size); - for (int _i90 = 0; _i90 < _list89.size; ++_i90) - { - long _elem91; - _elem91 = iprot.readI64(); - _val88.add(_elem91); - } - } - struct.currentBlocks.put(_key87, _val88); + long _elem71; + _elem71 = iprot.readI64(); + struct.currentBlocks.add(_elem71); } } struct.setCurrentBlocksIsSet(true); @@ -9220,8 +9165,7 @@ public static class worker_heartbeat_args implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -9231,15 +9175,13 @@ public static class worker_heartbeat_args implements org.apache.thrift.TBase removedBlockIds; // required - public Map> evictedBlockIds; // required + public List removedBlocks; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { WORKER_ID((short)1, "workerId"), USED_BYTES((short)2, "usedBytes"), - REMOVED_BLOCK_IDS((short)3, "removedBlockIds"), - EVICTED_BLOCK_IDS((short)4, "evictedBlockIds"); + REMOVED_BLOCKS((short)3, "removedBlocks"); private static final Map byName = new HashMap(); @@ -9258,10 +9200,8 @@ public static _Fields findByThriftId(int fieldId) { return WORKER_ID; case 2: // USED_BYTES return USED_BYTES; - case 3: // REMOVED_BLOCK_IDS - return REMOVED_BLOCK_IDS; - case 4: // EVICTED_BLOCK_IDS - return EVICTED_BLOCK_IDS; + case 3: // REMOVED_BLOCKS + return REMOVED_BLOCKS; default: return null; } @@ -9312,14 +9252,9 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.USED_BYTES, new org.apache.thrift.meta_data.FieldMetaData("usedBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.REMOVED_BLOCK_IDS, new org.apache.thrift.meta_data.FieldMetaData("removedBlockIds", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.REMOVED_BLOCKS, new org.apache.thrift.meta_data.FieldMetaData("removedBlocks", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); - tmpMap.put(_Fields.EVICTED_BLOCK_IDS, new org.apache.thrift.meta_data.FieldMetaData("evictedBlockIds", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64), - new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(worker_heartbeat_args.class, metaDataMap); } @@ -9330,16 +9265,14 @@ public worker_heartbeat_args() { public worker_heartbeat_args( long workerId, long usedBytes, - List removedBlockIds, - Map> evictedBlockIds) + List removedBlocks) { this(); this.workerId = workerId; setWorkerIdIsSet(true); this.usedBytes = usedBytes; setUsedBytesIsSet(true); - this.removedBlockIds = removedBlockIds; - this.evictedBlockIds = evictedBlockIds; + this.removedBlocks = removedBlocks; } /** @@ -9349,24 +9282,9 @@ public worker_heartbeat_args(worker_heartbeat_args other) { __isset_bitfield = other.__isset_bitfield; this.workerId = other.workerId; this.usedBytes = other.usedBytes; - if (other.isSetRemovedBlockIds()) { - List __this__removedBlockIds = new ArrayList(other.removedBlockIds); - this.removedBlockIds = __this__removedBlockIds; - } - if (other.isSetEvictedBlockIds()) { - Map> __this__evictedBlockIds = new HashMap>(other.evictedBlockIds.size()); - for (Map.Entry> other_element : other.evictedBlockIds.entrySet()) { - - Long other_element_key = other_element.getKey(); - List other_element_value = other_element.getValue(); - - Long __this__evictedBlockIds_copy_key = other_element_key; - - List __this__evictedBlockIds_copy_value = new ArrayList(other_element_value); - - __this__evictedBlockIds.put(__this__evictedBlockIds_copy_key, __this__evictedBlockIds_copy_value); - } - this.evictedBlockIds = __this__evictedBlockIds; + if (other.isSetRemovedBlocks()) { + List __this__removedBlocks = new ArrayList(other.removedBlocks); + this.removedBlocks = __this__removedBlocks; } } @@ -9380,8 +9298,7 @@ public void clear() { this.workerId = 0; setUsedBytesIsSet(false); this.usedBytes = 0; - this.removedBlockIds = null; - this.evictedBlockIds = null; + this.removedBlocks = null; } public long getWorkerId() { @@ -9430,77 +9347,42 @@ public void setUsedBytesIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USEDBYTES_ISSET_ID, value); } - public int getRemovedBlockIdsSize() { - return (this.removedBlockIds == null) ? 0 : this.removedBlockIds.size(); - } - - public java.util.Iterator getRemovedBlockIdsIterator() { - return (this.removedBlockIds == null) ? null : this.removedBlockIds.iterator(); - } - - public void addToRemovedBlockIds(long elem) { - if (this.removedBlockIds == null) { - this.removedBlockIds = new ArrayList(); - } - this.removedBlockIds.add(elem); - } - - public List getRemovedBlockIds() { - return this.removedBlockIds; - } - - public worker_heartbeat_args setRemovedBlockIds(List removedBlockIds) { - this.removedBlockIds = removedBlockIds; - return this; - } - - public void unsetRemovedBlockIds() { - this.removedBlockIds = null; - } - - /** Returns true if field removedBlockIds is set (has been assigned a value) and false otherwise */ - public boolean isSetRemovedBlockIds() { - return this.removedBlockIds != null; - } - - public void setRemovedBlockIdsIsSet(boolean value) { - if (!value) { - this.removedBlockIds = null; - } + public int getRemovedBlocksSize() { + return (this.removedBlocks == null) ? 0 : this.removedBlocks.size(); } - public int getEvictedBlockIdsSize() { - return (this.evictedBlockIds == null) ? 0 : this.evictedBlockIds.size(); + public java.util.Iterator getRemovedBlocksIterator() { + return (this.removedBlocks == null) ? null : this.removedBlocks.iterator(); } - public void putToEvictedBlockIds(long key, List val) { - if (this.evictedBlockIds == null) { - this.evictedBlockIds = new HashMap>(); + public void addToRemovedBlocks(long elem) { + if (this.removedBlocks == null) { + this.removedBlocks = new ArrayList(); } - this.evictedBlockIds.put(key, val); + this.removedBlocks.add(elem); } - public Map> getEvictedBlockIds() { - return this.evictedBlockIds; + public List getRemovedBlocks() { + return this.removedBlocks; } - public worker_heartbeat_args setEvictedBlockIds(Map> evictedBlockIds) { - this.evictedBlockIds = evictedBlockIds; + public worker_heartbeat_args setRemovedBlocks(List removedBlocks) { + this.removedBlocks = removedBlocks; return this; } - public void unsetEvictedBlockIds() { - this.evictedBlockIds = null; + public void unsetRemovedBlocks() { + this.removedBlocks = null; } - /** Returns true if field evictedBlockIds is set (has been assigned a value) and false otherwise */ - public boolean isSetEvictedBlockIds() { - return this.evictedBlockIds != null; + /** Returns true if field removedBlocks is set (has been assigned a value) and false otherwise */ + public boolean isSetRemovedBlocks() { + return this.removedBlocks != null; } - public void setEvictedBlockIdsIsSet(boolean value) { + public void setRemovedBlocksIsSet(boolean value) { if (!value) { - this.evictedBlockIds = null; + this.removedBlocks = null; } } @@ -9522,19 +9404,11 @@ public void setFieldValue(_Fields field, Object value) { } break; - case REMOVED_BLOCK_IDS: - if (value == null) { - unsetRemovedBlockIds(); - } else { - setRemovedBlockIds((List)value); - } - break; - - case EVICTED_BLOCK_IDS: + case REMOVED_BLOCKS: if (value == null) { - unsetEvictedBlockIds(); + unsetRemovedBlocks(); } else { - setEvictedBlockIds((Map>)value); + setRemovedBlocks((List)value); } break; @@ -9549,11 +9423,8 @@ public Object getFieldValue(_Fields field) { case USED_BYTES: return Long.valueOf(getUsedBytes()); - case REMOVED_BLOCK_IDS: - return getRemovedBlockIds(); - - case EVICTED_BLOCK_IDS: - return getEvictedBlockIds(); + case REMOVED_BLOCKS: + return getRemovedBlocks(); } throw new IllegalStateException(); @@ -9570,10 +9441,8 @@ public boolean isSet(_Fields field) { return isSetWorkerId(); case USED_BYTES: return isSetUsedBytes(); - case REMOVED_BLOCK_IDS: - return isSetRemovedBlockIds(); - case EVICTED_BLOCK_IDS: - return isSetEvictedBlockIds(); + case REMOVED_BLOCKS: + return isSetRemovedBlocks(); } throw new IllegalStateException(); } @@ -9609,21 +9478,12 @@ public boolean equals(worker_heartbeat_args that) { return false; } - boolean this_present_removedBlockIds = true && this.isSetRemovedBlockIds(); - boolean that_present_removedBlockIds = true && that.isSetRemovedBlockIds(); - if (this_present_removedBlockIds || that_present_removedBlockIds) { - if (!(this_present_removedBlockIds && that_present_removedBlockIds)) - return false; - if (!this.removedBlockIds.equals(that.removedBlockIds)) - return false; - } - - boolean this_present_evictedBlockIds = true && this.isSetEvictedBlockIds(); - boolean that_present_evictedBlockIds = true && that.isSetEvictedBlockIds(); - if (this_present_evictedBlockIds || that_present_evictedBlockIds) { - if (!(this_present_evictedBlockIds && that_present_evictedBlockIds)) + boolean this_present_removedBlocks = true && this.isSetRemovedBlocks(); + boolean that_present_removedBlocks = true && that.isSetRemovedBlocks(); + if (this_present_removedBlocks || that_present_removedBlocks) { + if (!(this_present_removedBlocks && that_present_removedBlocks)) return false; - if (!this.evictedBlockIds.equals(that.evictedBlockIds)) + if (!this.removedBlocks.equals(that.removedBlocks)) return false; } @@ -9663,22 +9523,12 @@ public int compareTo(worker_heartbeat_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRemovedBlockIds()).compareTo(other.isSetRemovedBlockIds()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRemovedBlockIds()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.removedBlockIds, other.removedBlockIds); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetEvictedBlockIds()).compareTo(other.isSetEvictedBlockIds()); + lastComparison = Boolean.valueOf(isSetRemovedBlocks()).compareTo(other.isSetRemovedBlocks()); if (lastComparison != 0) { return lastComparison; } - if (isSetEvictedBlockIds()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.evictedBlockIds, other.evictedBlockIds); + if (isSetRemovedBlocks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.removedBlocks, other.removedBlocks); if (lastComparison != 0) { return lastComparison; } @@ -9711,19 +9561,11 @@ public String toString() { sb.append(this.usedBytes); first = false; if (!first) sb.append(", "); - sb.append("removedBlockIds:"); - if (this.removedBlockIds == null) { - sb.append("null"); - } else { - sb.append(this.removedBlockIds); - } - first = false; - if (!first) sb.append(", "); - sb.append("evictedBlockIds:"); - if (this.evictedBlockIds == null) { + sb.append("removedBlocks:"); + if (this.removedBlocks == null) { sb.append("null"); } else { - sb.append(this.evictedBlockIds); + sb.append(this.removedBlocks); } first = false; sb.append(")"); @@ -9787,50 +9629,20 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_heartbeat_ar org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 3: // REMOVED_BLOCK_IDS + case 3: // REMOVED_BLOCKS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list92 = iprot.readListBegin(); - struct.removedBlockIds = new ArrayList(_list92.size); - for (int _i93 = 0; _i93 < _list92.size; ++_i93) + org.apache.thrift.protocol.TList _list72 = iprot.readListBegin(); + struct.removedBlocks = new ArrayList(_list72.size); + for (int _i73 = 0; _i73 < _list72.size; ++_i73) { - long _elem94; - _elem94 = iprot.readI64(); - struct.removedBlockIds.add(_elem94); + long _elem74; + _elem74 = iprot.readI64(); + struct.removedBlocks.add(_elem74); } iprot.readListEnd(); } - struct.setRemovedBlockIdsIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // EVICTED_BLOCK_IDS - if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { - { - org.apache.thrift.protocol.TMap _map95 = iprot.readMapBegin(); - struct.evictedBlockIds = new HashMap>(2*_map95.size); - for (int _i96 = 0; _i96 < _map95.size; ++_i96) - { - long _key97; - List _val98; - _key97 = iprot.readI64(); - { - org.apache.thrift.protocol.TList _list99 = iprot.readListBegin(); - _val98 = new ArrayList(_list99.size); - for (int _i100 = 0; _i100 < _list99.size; ++_i100) - { - long _elem101; - _elem101 = iprot.readI64(); - _val98.add(_elem101); - } - iprot.readListEnd(); - } - struct.evictedBlockIds.put(_key97, _val98); - } - iprot.readMapEnd(); - } - struct.setEvictedBlockIdsIsSet(true); + struct.setRemovedBlocksIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -9856,38 +9668,18 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, worker_heartbeat_a oprot.writeFieldBegin(USED_BYTES_FIELD_DESC); oprot.writeI64(struct.usedBytes); oprot.writeFieldEnd(); - if (struct.removedBlockIds != null) { - oprot.writeFieldBegin(REMOVED_BLOCK_IDS_FIELD_DESC); + if (struct.removedBlocks != null) { + oprot.writeFieldBegin(REMOVED_BLOCKS_FIELD_DESC); { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.removedBlockIds.size())); - for (long _iter102 : struct.removedBlockIds) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.removedBlocks.size())); + for (long _iter75 : struct.removedBlocks) { - oprot.writeI64(_iter102); + oprot.writeI64(_iter75); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } - if (struct.evictedBlockIds != null) { - oprot.writeFieldBegin(EVICTED_BLOCK_IDS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.LIST, struct.evictedBlockIds.size())); - for (Map.Entry> _iter103 : struct.evictedBlockIds.entrySet()) - { - oprot.writeI64(_iter103.getKey()); - { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter103.getValue().size())); - for (long _iter104 : _iter103.getValue()) - { - oprot.writeI64(_iter104); - } - oprot.writeListEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -9912,41 +9704,22 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_heartbeat_ar if (struct.isSetUsedBytes()) { optionals.set(1); } - if (struct.isSetRemovedBlockIds()) { + if (struct.isSetRemovedBlocks()) { optionals.set(2); } - if (struct.isSetEvictedBlockIds()) { - optionals.set(3); - } - oprot.writeBitSet(optionals, 4); + oprot.writeBitSet(optionals, 3); if (struct.isSetWorkerId()) { oprot.writeI64(struct.workerId); } if (struct.isSetUsedBytes()) { oprot.writeI64(struct.usedBytes); } - if (struct.isSetRemovedBlockIds()) { - { - oprot.writeI32(struct.removedBlockIds.size()); - for (long _iter105 : struct.removedBlockIds) - { - oprot.writeI64(_iter105); - } - } - } - if (struct.isSetEvictedBlockIds()) { + if (struct.isSetRemovedBlocks()) { { - oprot.writeI32(struct.evictedBlockIds.size()); - for (Map.Entry> _iter106 : struct.evictedBlockIds.entrySet()) + oprot.writeI32(struct.removedBlocks.size()); + for (long _iter76 : struct.removedBlocks) { - oprot.writeI64(_iter106.getKey()); - { - oprot.writeI32(_iter106.getValue().size()); - for (long _iter107 : _iter106.getValue()) - { - oprot.writeI64(_iter107); - } - } + oprot.writeI64(_iter76); } } } @@ -9955,7 +9728,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_heartbeat_ar @Override public void read(org.apache.thrift.protocol.TProtocol prot, worker_heartbeat_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(4); + BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { struct.workerId = iprot.readI64(); struct.setWorkerIdIsSet(true); @@ -9966,40 +9739,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, worker_heartbeat_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list108 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.removedBlockIds = new ArrayList(_list108.size); - for (int _i109 = 0; _i109 < _list108.size; ++_i109) - { - long _elem110; - _elem110 = iprot.readI64(); - struct.removedBlockIds.add(_elem110); - } - } - struct.setRemovedBlockIdsIsSet(true); - } - if (incoming.get(3)) { - { - org.apache.thrift.protocol.TMap _map111 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.LIST, iprot.readI32()); - struct.evictedBlockIds = new HashMap>(2*_map111.size); - for (int _i112 = 0; _i112 < _map111.size; ++_i112) + org.apache.thrift.protocol.TList _list77 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.removedBlocks = new ArrayList(_list77.size); + for (int _i78 = 0; _i78 < _list77.size; ++_i78) { - long _key113; - List _val114; - _key113 = iprot.readI64(); - { - org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _val114 = new ArrayList(_list115.size); - for (int _i116 = 0; _i116 < _list115.size; ++_i116) - { - long _elem117; - _elem117 = iprot.readI64(); - _val114.add(_elem117); - } - } - struct.evictedBlockIds.put(_key113, _val114); + long _elem79; + _elem79 = iprot.readI64(); + struct.removedBlocks.add(_elem79); } } - struct.setEvictedBlockIdsIsSet(true); + struct.setRemovedBlocksIsSet(true); } } } @@ -10472,9 +10221,8 @@ public static class worker_cacheBlock_args implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -10484,7 +10232,6 @@ public static class worker_cacheBlock_args implements org.apache.thrift.TBase byName = new HashMap(); @@ -10513,11 +10259,9 @@ public static _Fields findByThriftId(int fieldId) { return WORKER_ID; case 2: // WORKER_USED_BYTES return WORKER_USED_BYTES; - case 3: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 4: // BLOCK_ID + case 3: // BLOCK_ID return BLOCK_ID; - case 5: // LENGTH + case 4: // LENGTH return LENGTH; default: return null; @@ -10561,9 +10305,8 @@ public String getFieldName() { // isset id assignments private static final int __WORKERID_ISSET_ID = 0; private static final int __WORKERUSEDBYTES_ISSET_ID = 1; - private static final int __STORAGEDIRID_ISSET_ID = 2; - private static final int __BLOCKID_ISSET_ID = 3; - private static final int __LENGTH_ISSET_ID = 4; + private static final int __BLOCKID_ISSET_ID = 2; + private static final int __LENGTH_ISSET_ID = 3; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -10572,8 +10315,6 @@ public String getFieldName() { new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.WORKER_USED_BYTES, new org.apache.thrift.meta_data.FieldMetaData("workerUsedBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.BLOCK_ID, new org.apache.thrift.meta_data.FieldMetaData("blockId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.LENGTH, new org.apache.thrift.meta_data.FieldMetaData("length", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -10588,7 +10329,6 @@ public worker_cacheBlock_args() { public worker_cacheBlock_args( long workerId, long workerUsedBytes, - long storageDirId, long blockId, long length) { @@ -10597,8 +10337,6 @@ public worker_cacheBlock_args( setWorkerIdIsSet(true); this.workerUsedBytes = workerUsedBytes; setWorkerUsedBytesIsSet(true); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); this.blockId = blockId; setBlockIdIsSet(true); this.length = length; @@ -10612,7 +10350,6 @@ public worker_cacheBlock_args(worker_cacheBlock_args other) { __isset_bitfield = other.__isset_bitfield; this.workerId = other.workerId; this.workerUsedBytes = other.workerUsedBytes; - this.storageDirId = other.storageDirId; this.blockId = other.blockId; this.length = other.length; } @@ -10627,8 +10364,6 @@ public void clear() { this.workerId = 0; setWorkerUsedBytesIsSet(false); this.workerUsedBytes = 0; - setStorageDirIdIsSet(false); - this.storageDirId = 0; setBlockIdIsSet(false); this.blockId = 0; setLengthIsSet(false); @@ -10681,29 +10416,6 @@ public void setWorkerUsedBytesIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WORKERUSEDBYTES_ISSET_ID, value); } - public long getStorageDirId() { - return this.storageDirId; - } - - public worker_cacheBlock_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - public long getBlockId() { return this.blockId; } @@ -10768,14 +10480,6 @@ public void setFieldValue(_Fields field, Object value) { } break; - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - case BLOCK_ID: if (value == null) { unsetBlockId(); @@ -10803,9 +10507,6 @@ public Object getFieldValue(_Fields field) { case WORKER_USED_BYTES: return Long.valueOf(getWorkerUsedBytes()); - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - case BLOCK_ID: return Long.valueOf(getBlockId()); @@ -10827,8 +10528,6 @@ public boolean isSet(_Fields field) { return isSetWorkerId(); case WORKER_USED_BYTES: return isSetWorkerUsedBytes(); - case STORAGE_DIR_ID: - return isSetStorageDirId(); case BLOCK_ID: return isSetBlockId(); case LENGTH: @@ -10868,15 +10567,6 @@ public boolean equals(worker_cacheBlock_args that) { return false; } - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - boolean this_present_blockId = true; boolean that_present_blockId = true; if (this_present_blockId || that_present_blockId) { @@ -10931,16 +10621,6 @@ public int compareTo(worker_cacheBlock_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(isSetBlockId()).compareTo(other.isSetBlockId()); if (lastComparison != 0) { return lastComparison; @@ -10989,10 +10669,6 @@ public String toString() { sb.append(this.workerUsedBytes); first = false; if (!first) sb.append(", "); - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); sb.append("blockId:"); sb.append(this.blockId); first = false; @@ -11061,15 +10737,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_cacheBlock_a org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 3: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 4: // BLOCK_ID + case 3: // BLOCK_ID if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); @@ -11077,7 +10745,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_cacheBlock_a org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 5: // LENGTH + case 4: // LENGTH if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.length = iprot.readI64(); struct.setLengthIsSet(true); @@ -11106,9 +10774,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, worker_cacheBlock_ oprot.writeFieldBegin(WORKER_USED_BYTES_FIELD_DESC); oprot.writeI64(struct.workerUsedBytes); oprot.writeFieldEnd(); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); oprot.writeFieldBegin(BLOCK_ID_FIELD_DESC); oprot.writeI64(struct.blockId); oprot.writeFieldEnd(); @@ -11139,25 +10804,19 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_cacheBlock_a if (struct.isSetWorkerUsedBytes()) { optionals.set(1); } - if (struct.isSetStorageDirId()) { - optionals.set(2); - } if (struct.isSetBlockId()) { - optionals.set(3); + optionals.set(2); } if (struct.isSetLength()) { - optionals.set(4); + optionals.set(3); } - oprot.writeBitSet(optionals, 5); + oprot.writeBitSet(optionals, 4); if (struct.isSetWorkerId()) { oprot.writeI64(struct.workerId); } if (struct.isSetWorkerUsedBytes()) { oprot.writeI64(struct.workerUsedBytes); } - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } if (struct.isSetBlockId()) { oprot.writeI64(struct.blockId); } @@ -11169,7 +10828,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_cacheBlock_a @Override public void read(org.apache.thrift.protocol.TProtocol prot, worker_cacheBlock_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(5); + BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { struct.workerId = iprot.readI64(); struct.setWorkerIdIsSet(true); @@ -11179,14 +10838,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, worker_cacheBlock_ar struct.setWorkerUsedBytesIsSet(true); } if (incoming.get(2)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } - if (incoming.get(3)) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); } - if (incoming.get(4)) { + if (incoming.get(3)) { struct.length = iprot.readI64(); struct.setLengthIsSet(true); } @@ -12307,13 +11962,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_getPinIdList case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.SET) { { - org.apache.thrift.protocol.TSet _set118 = iprot.readSetBegin(); - struct.success = new HashSet(2*_set118.size); - for (int _i119 = 0; _i119 < _set118.size; ++_i119) + org.apache.thrift.protocol.TSet _set80 = iprot.readSetBegin(); + struct.success = new HashSet(2*_set80.size); + for (int _i81 = 0; _i81 < _set80.size; ++_i81) { - int _elem120; - _elem120 = iprot.readI32(); - struct.success.add(_elem120); + int _elem82; + _elem82 = iprot.readI32(); + struct.success.add(_elem82); } iprot.readSetEnd(); } @@ -12341,9 +11996,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, worker_getPinIdLis oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, struct.success.size())); - for (int _iter121 : struct.success) + for (int _iter83 : struct.success) { - oprot.writeI32(_iter121); + oprot.writeI32(_iter83); } oprot.writeSetEnd(); } @@ -12374,9 +12029,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_getPinIdList if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (int _iter122 : struct.success) + for (int _iter84 : struct.success) { - oprot.writeI32(_iter122); + oprot.writeI32(_iter84); } } } @@ -12388,13 +12043,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, worker_getPinIdList_ BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TSet _set123 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.success = new HashSet(2*_set123.size); - for (int _i124 = 0; _i124 < _set123.size; ++_i124) + org.apache.thrift.protocol.TSet _set85 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.success = new HashSet(2*_set85.size); + for (int _i86 = 0; _i86 < _set85.size; ++_i86) { - int _elem125; - _elem125 = iprot.readI32(); - struct.success.add(_elem125); + int _elem87; + _elem87 = iprot.readI32(); + struct.success.add(_elem87); } } struct.setSuccessIsSet(true); @@ -12956,13 +12611,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, worker_getPriorityD case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list126 = iprot.readListBegin(); - struct.success = new ArrayList(_list126.size); - for (int _i127 = 0; _i127 < _list126.size; ++_i127) + org.apache.thrift.protocol.TList _list88 = iprot.readListBegin(); + struct.success = new ArrayList(_list88.size); + for (int _i89 = 0; _i89 < _list88.size; ++_i89) { - int _elem128; - _elem128 = iprot.readI32(); - struct.success.add(_elem128); + int _elem90; + _elem90 = iprot.readI32(); + struct.success.add(_elem90); } iprot.readListEnd(); } @@ -12990,9 +12645,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, worker_getPriority oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.success.size())); - for (int _iter129 : struct.success) + for (int _iter91 : struct.success) { - oprot.writeI32(_iter129); + oprot.writeI32(_iter91); } oprot.writeListEnd(); } @@ -13023,9 +12678,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, worker_getPriorityD if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (int _iter130 : struct.success) + for (int _iter92 : struct.success) { - oprot.writeI32(_iter130); + oprot.writeI32(_iter92); } } } @@ -13037,13 +12692,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, worker_getPriorityDe BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.success = new ArrayList(_list131.size); - for (int _i132 = 0; _i132 < _list131.size; ++_i132) + org.apache.thrift.protocol.TList _list93 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.success = new ArrayList(_list93.size); + for (int _i94 = 0; _i94 < _list93.size; ++_i94) { - int _elem133; - _elem133 = iprot.readI32(); - struct.success.add(_elem133); + int _elem95; + _elem95 = iprot.readI32(); + struct.success.add(_elem95); } } struct.setSuccessIsSet(true); @@ -14005,13 +13660,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, user_createDependen case 1: // PARENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); - struct.parents = new ArrayList(_list134.size); - for (int _i135 = 0; _i135 < _list134.size; ++_i135) + org.apache.thrift.protocol.TList _list96 = iprot.readListBegin(); + struct.parents = new ArrayList(_list96.size); + for (int _i97 = 0; _i97 < _list96.size; ++_i97) { - String _elem136; - _elem136 = iprot.readString(); - struct.parents.add(_elem136); + String _elem98; + _elem98 = iprot.readString(); + struct.parents.add(_elem98); } iprot.readListEnd(); } @@ -14023,13 +13678,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, user_createDependen case 2: // CHILDREN if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list137 = iprot.readListBegin(); - struct.children = new ArrayList(_list137.size); - for (int _i138 = 0; _i138 < _list137.size; ++_i138) + org.apache.thrift.protocol.TList _list99 = iprot.readListBegin(); + struct.children = new ArrayList(_list99.size); + for (int _i100 = 0; _i100 < _list99.size; ++_i100) { - String _elem139; - _elem139 = iprot.readString(); - struct.children.add(_elem139); + String _elem101; + _elem101 = iprot.readString(); + struct.children.add(_elem101); } iprot.readListEnd(); } @@ -14049,13 +13704,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, user_createDependen case 4: // DATA if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list140 = iprot.readListBegin(); - struct.data = new ArrayList(_list140.size); - for (int _i141 = 0; _i141 < _list140.size; ++_i141) + org.apache.thrift.protocol.TList _list102 = iprot.readListBegin(); + struct.data = new ArrayList(_list102.size); + for (int _i103 = 0; _i103 < _list102.size; ++_i103) { - ByteBuffer _elem142; - _elem142 = iprot.readBinary(); - struct.data.add(_elem142); + ByteBuffer _elem104; + _elem104 = iprot.readBinary(); + struct.data.add(_elem104); } iprot.readListEnd(); } @@ -14123,9 +13778,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, user_createDepende oprot.writeFieldBegin(PARENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.parents.size())); - for (String _iter143 : struct.parents) + for (String _iter105 : struct.parents) { - oprot.writeString(_iter143); + oprot.writeString(_iter105); } oprot.writeListEnd(); } @@ -14135,9 +13790,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, user_createDepende oprot.writeFieldBegin(CHILDREN_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.children.size())); - for (String _iter144 : struct.children) + for (String _iter106 : struct.children) { - oprot.writeString(_iter144); + oprot.writeString(_iter106); } oprot.writeListEnd(); } @@ -14152,9 +13807,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, user_createDepende oprot.writeFieldBegin(DATA_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.data.size())); - for (ByteBuffer _iter145 : struct.data) + for (ByteBuffer _iter107 : struct.data) { - oprot.writeBinary(_iter145); + oprot.writeBinary(_iter107); } oprot.writeListEnd(); } @@ -14230,18 +13885,18 @@ public void write(org.apache.thrift.protocol.TProtocol prot, user_createDependen if (struct.isSetParents()) { { oprot.writeI32(struct.parents.size()); - for (String _iter146 : struct.parents) + for (String _iter108 : struct.parents) { - oprot.writeString(_iter146); + oprot.writeString(_iter108); } } } if (struct.isSetChildren()) { { oprot.writeI32(struct.children.size()); - for (String _iter147 : struct.children) + for (String _iter109 : struct.children) { - oprot.writeString(_iter147); + oprot.writeString(_iter109); } } } @@ -14251,9 +13906,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, user_createDependen if (struct.isSetData()) { { oprot.writeI32(struct.data.size()); - for (ByteBuffer _iter148 : struct.data) + for (ByteBuffer _iter110 : struct.data) { - oprot.writeBinary(_iter148); + oprot.writeBinary(_iter110); } } } @@ -14280,26 +13935,26 @@ public void read(org.apache.thrift.protocol.TProtocol prot, user_createDependenc BitSet incoming = iprot.readBitSet(9); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list149 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.parents = new ArrayList(_list149.size); - for (int _i150 = 0; _i150 < _list149.size; ++_i150) + org.apache.thrift.protocol.TList _list111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.parents = new ArrayList(_list111.size); + for (int _i112 = 0; _i112 < _list111.size; ++_i112) { - String _elem151; - _elem151 = iprot.readString(); - struct.parents.add(_elem151); + String _elem113; + _elem113 = iprot.readString(); + struct.parents.add(_elem113); } } struct.setParentsIsSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list152 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.children = new ArrayList(_list152.size); - for (int _i153 = 0; _i153 < _list152.size; ++_i153) + org.apache.thrift.protocol.TList _list114 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.children = new ArrayList(_list114.size); + for (int _i115 = 0; _i115 < _list114.size; ++_i115) { - String _elem154; - _elem154 = iprot.readString(); - struct.children.add(_elem154); + String _elem116; + _elem116 = iprot.readString(); + struct.children.add(_elem116); } } struct.setChildrenIsSet(true); @@ -14310,13 +13965,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, user_createDependenc } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list155 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.data = new ArrayList(_list155.size); - for (int _i156 = 0; _i156 < _list155.size; ++_i156) + org.apache.thrift.protocol.TList _list117 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.data = new ArrayList(_list117.size); + for (int _i118 = 0; _i118 < _list117.size; ++_i118) { - ByteBuffer _elem157; - _elem157 = iprot.readBinary(); - struct.data.add(_elem157); + ByteBuffer _elem119; + _elem119 = iprot.readBinary(); + struct.data.add(_elem119); } } struct.setDataIsSet(true); @@ -25623,14 +25278,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, user_getFileBlocks_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list158 = iprot.readListBegin(); - struct.success = new ArrayList(_list158.size); - for (int _i159 = 0; _i159 < _list158.size; ++_i159) + org.apache.thrift.protocol.TList _list120 = iprot.readListBegin(); + struct.success = new ArrayList(_list120.size); + for (int _i121 = 0; _i121 < _list120.size; ++_i121) { - ClientBlockInfo _elem160; - _elem160 = new ClientBlockInfo(); - _elem160.read(iprot); - struct.success.add(_elem160); + ClientBlockInfo _elem122; + _elem122 = new ClientBlockInfo(); + _elem122.read(iprot); + struct.success.add(_elem122); } iprot.readListEnd(); } @@ -25676,9 +25331,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, user_getFileBlocks oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (ClientBlockInfo _iter161 : struct.success) + for (ClientBlockInfo _iter123 : struct.success) { - _iter161.write(oprot); + _iter123.write(oprot); } oprot.writeListEnd(); } @@ -25725,9 +25380,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, user_getFileBlocks_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ClientBlockInfo _iter162 : struct.success) + for (ClientBlockInfo _iter124 : struct.success) { - _iter162.write(oprot); + _iter124.write(oprot); } } } @@ -25745,14 +25400,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, user_getFileBlocks_r BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list163 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list163.size); - for (int _i164 = 0; _i164 < _list163.size; ++_i164) + org.apache.thrift.protocol.TList _list125 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list125.size); + for (int _i126 = 0; _i126 < _list125.size; ++_i126) { - ClientBlockInfo _elem165; - _elem165 = new ClientBlockInfo(); - _elem165.read(iprot); - struct.success.add(_elem165); + ClientBlockInfo _elem127; + _elem127 = new ClientBlockInfo(); + _elem127.read(iprot); + struct.success.add(_elem127); } } struct.setSuccessIsSet(true); diff --git a/core/src/main/java/tachyon/thrift/OutOfSpaceException.java b/core/src/main/java/tachyon/thrift/OutOfMemoryForPinFileException.java similarity index 81% rename from core/src/main/java/tachyon/thrift/OutOfSpaceException.java rename to core/src/main/java/tachyon/thrift/OutOfMemoryForPinFileException.java index 13f6343c502e..d84631ae5428 100644 --- a/core/src/main/java/tachyon/thrift/OutOfSpaceException.java +++ b/core/src/main/java/tachyon/thrift/OutOfMemoryForPinFileException.java @@ -32,15 +32,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class OutOfSpaceException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OutOfSpaceException"); +public class OutOfMemoryForPinFileException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OutOfMemoryForPinFileException"); private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new OutOfSpaceExceptionStandardSchemeFactory()); - schemes.put(TupleScheme.class, new OutOfSpaceExceptionTupleSchemeFactory()); + schemes.put(StandardScheme.class, new OutOfMemoryForPinFileExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OutOfMemoryForPinFileExceptionTupleSchemeFactory()); } public String message; // required @@ -110,13 +110,13 @@ public String getFieldName() { tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OutOfSpaceException.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OutOfMemoryForPinFileException.class, metaDataMap); } - public OutOfSpaceException() { + public OutOfMemoryForPinFileException() { } - public OutOfSpaceException( + public OutOfMemoryForPinFileException( String message) { this(); @@ -126,14 +126,14 @@ public OutOfSpaceException( /** * Performs a deep copy on other. */ - public OutOfSpaceException(OutOfSpaceException other) { + public OutOfMemoryForPinFileException(OutOfMemoryForPinFileException other) { if (other.isSetMessage()) { this.message = other.message; } } - public OutOfSpaceException deepCopy() { - return new OutOfSpaceException(this); + public OutOfMemoryForPinFileException deepCopy() { + return new OutOfMemoryForPinFileException(this); } @Override @@ -145,7 +145,7 @@ public String getMessage() { return this.message; } - public OutOfSpaceException setMessage(String message) { + public OutOfMemoryForPinFileException setMessage(String message) { this.message = message; return this; } @@ -204,12 +204,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof OutOfSpaceException) - return this.equals((OutOfSpaceException)that); + if (that instanceof OutOfMemoryForPinFileException) + return this.equals((OutOfMemoryForPinFileException)that); return false; } - public boolean equals(OutOfSpaceException that) { + public boolean equals(OutOfMemoryForPinFileException that) { if (that == null) return false; @@ -231,7 +231,7 @@ public int hashCode() { } @Override - public int compareTo(OutOfSpaceException other) { + public int compareTo(OutOfMemoryForPinFileException other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -265,7 +265,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("OutOfSpaceException("); + StringBuilder sb = new StringBuilder("OutOfMemoryForPinFileException("); boolean first = true; sb.append("message:"); @@ -300,15 +300,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class OutOfSpaceExceptionStandardSchemeFactory implements SchemeFactory { - public OutOfSpaceExceptionStandardScheme getScheme() { - return new OutOfSpaceExceptionStandardScheme(); + private static class OutOfMemoryForPinFileExceptionStandardSchemeFactory implements SchemeFactory { + public OutOfMemoryForPinFileExceptionStandardScheme getScheme() { + return new OutOfMemoryForPinFileExceptionStandardScheme(); } } - private static class OutOfSpaceExceptionStandardScheme extends StandardScheme { + private static class OutOfMemoryForPinFileExceptionStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, OutOfSpaceException struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, OutOfMemoryForPinFileException struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -337,7 +337,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, OutOfSpaceException struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, OutOfSpaceException struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, OutOfMemoryForPinFileException struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -352,16 +352,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, OutOfSpaceExceptio } - private static class OutOfSpaceExceptionTupleSchemeFactory implements SchemeFactory { - public OutOfSpaceExceptionTupleScheme getScheme() { - return new OutOfSpaceExceptionTupleScheme(); + private static class OutOfMemoryForPinFileExceptionTupleSchemeFactory implements SchemeFactory { + public OutOfMemoryForPinFileExceptionTupleScheme getScheme() { + return new OutOfMemoryForPinFileExceptionTupleScheme(); } } - private static class OutOfSpaceExceptionTupleScheme extends TupleScheme { + private static class OutOfMemoryForPinFileExceptionTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, OutOfSpaceException struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, OutOfMemoryForPinFileException struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetMessage()) { @@ -374,7 +374,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, OutOfSpaceException } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, OutOfSpaceException struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, OutOfMemoryForPinFileException struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { diff --git a/core/src/main/java/tachyon/thrift/WorkerService.java b/core/src/main/java/tachyon/thrift/WorkerService.java index 5de020fbc405..c419a6c4eab2 100644 --- a/core/src/main/java/tachyon/thrift/WorkerService.java +++ b/core/src/main/java/tachyon/thrift/WorkerService.java @@ -36,31 +36,27 @@ public class WorkerService { public interface Iface { - public void accessBlock(long storageDirId, long blockId) throws org.apache.thrift.TException; + public void accessBlock(long blockId) throws org.apache.thrift.TException; public void addCheckpoint(long userId, int fileId) throws FileDoesNotExistException, SuspectedFileSizeException, FailedToCheckpointException, BlockInfoException, org.apache.thrift.TException; public boolean asyncCheckpoint(int fileId) throws TachyonException, org.apache.thrift.TException; - public void cacheBlock(long userId, long storageDirId, long blockId) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException; + public void cacheBlock(long userId, long blockId) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException; - public ClientLocationInfo getLocalBlockLocation(long blockId) throws FileDoesNotExistException, org.apache.thrift.TException; + public String getDataFolder() throws org.apache.thrift.TException; - public String getUserLocalTempFolder(long userId, long storageDirId) throws org.apache.thrift.TException; + public String getUserTempFolder(long userId) throws org.apache.thrift.TException; public String getUserUfsTempFolder(long userId) throws org.apache.thrift.TException; - public ClientLocationInfo lockBlock(long blockId, long userId) throws FileDoesNotExistException, org.apache.thrift.TException; + public void lockBlock(long blockId, long userId) throws org.apache.thrift.TException; - public boolean promoteBlock(long userId, long blockId) throws org.apache.thrift.TException; + public void returnSpace(long userId, long returnedBytes) throws org.apache.thrift.TException; - public void returnSpace(long userId, long storageDirId, long returnedBytes) throws org.apache.thrift.TException; + public boolean requestSpace(long userId, long requestBytes) throws org.apache.thrift.TException; - public ClientLocationInfo requestSpace(long userId, long requestBytes) throws OutOfSpaceException, org.apache.thrift.TException; - - public boolean requestSpaceInPlace(long userId, long storageDirId, long requestBytes) throws org.apache.thrift.TException; - - public boolean unlockBlock(long blockId, long userId) throws org.apache.thrift.TException; + public void unlockBlock(long blockId, long userId) throws org.apache.thrift.TException; public void userHeartbeat(long userId) throws org.apache.thrift.TException; @@ -68,30 +64,26 @@ public interface Iface { public interface AsyncIface { - public void accessBlock(long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void accessBlock(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void addCheckpoint(long userId, int fileId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void asyncCheckpoint(int fileId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void cacheBlock(long userId, long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void cacheBlock(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getLocalBlockLocation(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getDataFolder(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getUserLocalTempFolder(long userId, long storageDirId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getUserTempFolder(long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void getUserUfsTempFolder(long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void lockBlock(long blockId, long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void promoteBlock(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - - public void returnSpace(long userId, long storageDirId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void returnSpace(long userId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void requestSpace(long userId, long requestBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void requestSpaceInPlace(long userId, long storageDirId, long requestBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void unlockBlock(long blockId, long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void userHeartbeat(long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -118,16 +110,15 @@ public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.prot super(iprot, oprot); } - public void accessBlock(long storageDirId, long blockId) throws org.apache.thrift.TException + public void accessBlock(long blockId) throws org.apache.thrift.TException { - send_accessBlock(storageDirId, blockId); + send_accessBlock(blockId); recv_accessBlock(); } - public void send_accessBlock(long storageDirId, long blockId) throws org.apache.thrift.TException + public void send_accessBlock(long blockId) throws org.apache.thrift.TException { accessBlock_args args = new accessBlock_args(); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); sendBase("accessBlock", args); } @@ -198,17 +189,16 @@ public boolean recv_asyncCheckpoint() throws TachyonException, org.apache.thrift throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "asyncCheckpoint failed: unknown result"); } - public void cacheBlock(long userId, long storageDirId, long blockId) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException + public void cacheBlock(long userId, long blockId) throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, org.apache.thrift.TException { - send_cacheBlock(userId, storageDirId, blockId); + send_cacheBlock(userId, blockId); recv_cacheBlock(); } - public void send_cacheBlock(long userId, long storageDirId, long blockId) throws org.apache.thrift.TException + public void send_cacheBlock(long userId, long blockId) throws org.apache.thrift.TException { cacheBlock_args args = new cacheBlock_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); sendBase("cacheBlock", args); } @@ -229,54 +219,49 @@ public void recv_cacheBlock() throws FileDoesNotExistException, SuspectedFileSiz return; } - public ClientLocationInfo getLocalBlockLocation(long blockId) throws FileDoesNotExistException, org.apache.thrift.TException + public String getDataFolder() throws org.apache.thrift.TException { - send_getLocalBlockLocation(blockId); - return recv_getLocalBlockLocation(); + send_getDataFolder(); + return recv_getDataFolder(); } - public void send_getLocalBlockLocation(long blockId) throws org.apache.thrift.TException + public void send_getDataFolder() throws org.apache.thrift.TException { - getLocalBlockLocation_args args = new getLocalBlockLocation_args(); - args.setBlockId(blockId); - sendBase("getLocalBlockLocation", args); + getDataFolder_args args = new getDataFolder_args(); + sendBase("getDataFolder", args); } - public ClientLocationInfo recv_getLocalBlockLocation() throws FileDoesNotExistException, org.apache.thrift.TException + public String recv_getDataFolder() throws org.apache.thrift.TException { - getLocalBlockLocation_result result = new getLocalBlockLocation_result(); - receiveBase(result, "getLocalBlockLocation"); + getDataFolder_result result = new getDataFolder_result(); + receiveBase(result, "getDataFolder"); if (result.isSetSuccess()) { return result.success; } - if (result.eP != null) { - throw result.eP; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLocalBlockLocation failed: unknown result"); + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getDataFolder failed: unknown result"); } - public String getUserLocalTempFolder(long userId, long storageDirId) throws org.apache.thrift.TException + public String getUserTempFolder(long userId) throws org.apache.thrift.TException { - send_getUserLocalTempFolder(userId, storageDirId); - return recv_getUserLocalTempFolder(); + send_getUserTempFolder(userId); + return recv_getUserTempFolder(); } - public void send_getUserLocalTempFolder(long userId, long storageDirId) throws org.apache.thrift.TException + public void send_getUserTempFolder(long userId) throws org.apache.thrift.TException { - getUserLocalTempFolder_args args = new getUserLocalTempFolder_args(); + getUserTempFolder_args args = new getUserTempFolder_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); - sendBase("getUserLocalTempFolder", args); + sendBase("getUserTempFolder", args); } - public String recv_getUserLocalTempFolder() throws org.apache.thrift.TException + public String recv_getUserTempFolder() throws org.apache.thrift.TException { - getUserLocalTempFolder_result result = new getUserLocalTempFolder_result(); - receiveBase(result, "getUserLocalTempFolder"); + getUserTempFolder_result result = new getUserTempFolder_result(); + receiveBase(result, "getUserTempFolder"); if (result.isSetSuccess()) { return result.success; } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserLocalTempFolder failed: unknown result"); + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserTempFolder failed: unknown result"); } public String getUserUfsTempFolder(long userId) throws org.apache.thrift.TException @@ -302,10 +287,10 @@ public String recv_getUserUfsTempFolder() throws org.apache.thrift.TException throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserUfsTempFolder failed: unknown result"); } - public ClientLocationInfo lockBlock(long blockId, long userId) throws FileDoesNotExistException, org.apache.thrift.TException + public void lockBlock(long blockId, long userId) throws org.apache.thrift.TException { send_lockBlock(blockId, userId); - return recv_lockBlock(); + recv_lockBlock(); } public void send_lockBlock(long blockId, long userId) throws org.apache.thrift.TException @@ -316,54 +301,23 @@ public void send_lockBlock(long blockId, long userId) throws org.apache.thrift.T sendBase("lockBlock", args); } - public ClientLocationInfo recv_lockBlock() throws FileDoesNotExistException, org.apache.thrift.TException + public void recv_lockBlock() throws org.apache.thrift.TException { lockBlock_result result = new lockBlock_result(); receiveBase(result, "lockBlock"); - if (result.isSetSuccess()) { - return result.success; - } - if (result.eP != null) { - throw result.eP; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "lockBlock failed: unknown result"); - } - - public boolean promoteBlock(long userId, long blockId) throws org.apache.thrift.TException - { - send_promoteBlock(userId, blockId); - return recv_promoteBlock(); - } - - public void send_promoteBlock(long userId, long blockId) throws org.apache.thrift.TException - { - promoteBlock_args args = new promoteBlock_args(); - args.setUserId(userId); - args.setBlockId(blockId); - sendBase("promoteBlock", args); - } - - public boolean recv_promoteBlock() throws org.apache.thrift.TException - { - promoteBlock_result result = new promoteBlock_result(); - receiveBase(result, "promoteBlock"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "promoteBlock failed: unknown result"); + return; } - public void returnSpace(long userId, long storageDirId, long returnedBytes) throws org.apache.thrift.TException + public void returnSpace(long userId, long returnedBytes) throws org.apache.thrift.TException { - send_returnSpace(userId, storageDirId, returnedBytes); + send_returnSpace(userId, returnedBytes); recv_returnSpace(); } - public void send_returnSpace(long userId, long storageDirId, long returnedBytes) throws org.apache.thrift.TException + public void send_returnSpace(long userId, long returnedBytes) throws org.apache.thrift.TException { returnSpace_args args = new returnSpace_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); args.setReturnedBytes(returnedBytes); sendBase("returnSpace", args); } @@ -375,7 +329,7 @@ public void recv_returnSpace() throws org.apache.thrift.TException return; } - public ClientLocationInfo requestSpace(long userId, long requestBytes) throws OutOfSpaceException, org.apache.thrift.TException + public boolean requestSpace(long userId, long requestBytes) throws org.apache.thrift.TException { send_requestSpace(userId, requestBytes); return recv_requestSpace(); @@ -389,48 +343,20 @@ public void send_requestSpace(long userId, long requestBytes) throws org.apache. sendBase("requestSpace", args); } - public ClientLocationInfo recv_requestSpace() throws OutOfSpaceException, org.apache.thrift.TException + public boolean recv_requestSpace() throws org.apache.thrift.TException { requestSpace_result result = new requestSpace_result(); receiveBase(result, "requestSpace"); if (result.isSetSuccess()) { return result.success; } - if (result.eP != null) { - throw result.eP; - } throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "requestSpace failed: unknown result"); } - public boolean requestSpaceInPlace(long userId, long storageDirId, long requestBytes) throws org.apache.thrift.TException - { - send_requestSpaceInPlace(userId, storageDirId, requestBytes); - return recv_requestSpaceInPlace(); - } - - public void send_requestSpaceInPlace(long userId, long storageDirId, long requestBytes) throws org.apache.thrift.TException - { - requestSpaceInPlace_args args = new requestSpaceInPlace_args(); - args.setUserId(userId); - args.setStorageDirId(storageDirId); - args.setRequestBytes(requestBytes); - sendBase("requestSpaceInPlace", args); - } - - public boolean recv_requestSpaceInPlace() throws org.apache.thrift.TException - { - requestSpaceInPlace_result result = new requestSpaceInPlace_result(); - receiveBase(result, "requestSpaceInPlace"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "requestSpaceInPlace failed: unknown result"); - } - - public boolean unlockBlock(long blockId, long userId) throws org.apache.thrift.TException + public void unlockBlock(long blockId, long userId) throws org.apache.thrift.TException { send_unlockBlock(blockId, userId); - return recv_unlockBlock(); + recv_unlockBlock(); } public void send_unlockBlock(long blockId, long userId) throws org.apache.thrift.TException @@ -441,14 +367,11 @@ public void send_unlockBlock(long blockId, long userId) throws org.apache.thrift sendBase("unlockBlock", args); } - public boolean recv_unlockBlock() throws org.apache.thrift.TException + public void recv_unlockBlock() throws org.apache.thrift.TException { unlockBlock_result result = new unlockBlock_result(); receiveBase(result, "unlockBlock"); - if (result.isSetSuccess()) { - return result.success; - } - throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "unlockBlock failed: unknown result"); + return; } public void userHeartbeat(long userId) throws org.apache.thrift.TException @@ -489,26 +412,23 @@ public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, super(protocolFactory, clientManager, transport); } - public void accessBlock(long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void accessBlock(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - accessBlock_call method_call = new accessBlock_call(storageDirId, blockId, resultHandler, this, ___protocolFactory, ___transport); + accessBlock_call method_call = new accessBlock_call(blockId, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class accessBlock_call extends org.apache.thrift.async.TAsyncMethodCall { - private long storageDirId; private long blockId; - public accessBlock_call(long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public accessBlock_call(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); - this.storageDirId = storageDirId; this.blockId = blockId; } public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("accessBlock", org.apache.thrift.protocol.TMessageType.CALL, 0)); accessBlock_args args = new accessBlock_args(); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); args.write(prot); prot.writeMessageEnd(); @@ -591,21 +511,19 @@ public boolean getResult() throws TachyonException, org.apache.thrift.TException } } - public void cacheBlock(long userId, long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void cacheBlock(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - cacheBlock_call method_call = new cacheBlock_call(userId, storageDirId, blockId, resultHandler, this, ___protocolFactory, ___transport); + cacheBlock_call method_call = new cacheBlock_call(userId, blockId, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class cacheBlock_call extends org.apache.thrift.async.TAsyncMethodCall { private long userId; - private long storageDirId; private long blockId; - public cacheBlock_call(long userId, long storageDirId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public cacheBlock_call(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.userId = userId; - this.storageDirId = storageDirId; this.blockId = blockId; } @@ -613,7 +531,6 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cacheBlock", org.apache.thrift.protocol.TMessageType.CALL, 0)); cacheBlock_args args = new cacheBlock_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); args.setBlockId(blockId); args.write(prot); prot.writeMessageEnd(); @@ -629,59 +546,53 @@ public void getResult() throws FileDoesNotExistException, SuspectedFileSizeExcep } } - public void getLocalBlockLocation(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getDataFolder(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - getLocalBlockLocation_call method_call = new getLocalBlockLocation_call(blockId, resultHandler, this, ___protocolFactory, ___transport); + getDataFolder_call method_call = new getDataFolder_call(resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getLocalBlockLocation_call extends org.apache.thrift.async.TAsyncMethodCall { - private long blockId; - public getLocalBlockLocation_call(long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public static class getDataFolder_call extends org.apache.thrift.async.TAsyncMethodCall { + public getDataFolder_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); - this.blockId = blockId; } public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getLocalBlockLocation", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getLocalBlockLocation_args args = new getLocalBlockLocation_args(); - args.setBlockId(blockId); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getDataFolder", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getDataFolder_args args = new getDataFolder_args(); args.write(prot); prot.writeMessageEnd(); } - public ClientLocationInfo getResult() throws FileDoesNotExistException, org.apache.thrift.TException { + public String getResult() throws org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getLocalBlockLocation(); + return (new Client(prot)).recv_getDataFolder(); } } - public void getUserLocalTempFolder(long userId, long storageDirId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getUserTempFolder(long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - getUserLocalTempFolder_call method_call = new getUserLocalTempFolder_call(userId, storageDirId, resultHandler, this, ___protocolFactory, ___transport); + getUserTempFolder_call method_call = new getUserTempFolder_call(userId, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } - public static class getUserLocalTempFolder_call extends org.apache.thrift.async.TAsyncMethodCall { + public static class getUserTempFolder_call extends org.apache.thrift.async.TAsyncMethodCall { private long userId; - private long storageDirId; - public getUserLocalTempFolder_call(long userId, long storageDirId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public getUserTempFolder_call(long userId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.userId = userId; - this.storageDirId = storageDirId; } public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getUserLocalTempFolder", org.apache.thrift.protocol.TMessageType.CALL, 0)); - getUserLocalTempFolder_args args = new getUserLocalTempFolder_args(); + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getUserTempFolder", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getUserTempFolder_args args = new getUserTempFolder_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); args.write(prot); prot.writeMessageEnd(); } @@ -692,7 +603,7 @@ public String getResult() throws org.apache.thrift.TException { } org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getUserLocalTempFolder(); + return (new Client(prot)).recv_getUserTempFolder(); } } @@ -753,66 +664,29 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public ClientLocationInfo getResult() throws FileDoesNotExistException, org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_lockBlock(); - } - } - - public void promoteBlock(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - promoteBlock_call method_call = new promoteBlock_call(userId, blockId, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class promoteBlock_call extends org.apache.thrift.async.TAsyncMethodCall { - private long userId; - private long blockId; - public promoteBlock_call(long userId, long blockId, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.userId = userId; - this.blockId = blockId; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("promoteBlock", org.apache.thrift.protocol.TMessageType.CALL, 0)); - promoteBlock_args args = new promoteBlock_args(); - args.setUserId(userId); - args.setBlockId(blockId); - args.write(prot); - prot.writeMessageEnd(); - } - - public boolean getResult() throws org.apache.thrift.TException { + public void getResult() throws org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_promoteBlock(); + (new Client(prot)).recv_lockBlock(); } } - public void returnSpace(long userId, long storageDirId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void returnSpace(long userId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); - returnSpace_call method_call = new returnSpace_call(userId, storageDirId, returnedBytes, resultHandler, this, ___protocolFactory, ___transport); + returnSpace_call method_call = new returnSpace_call(userId, returnedBytes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; ___manager.call(method_call); } public static class returnSpace_call extends org.apache.thrift.async.TAsyncMethodCall { private long userId; - private long storageDirId; private long returnedBytes; - public returnSpace_call(long userId, long storageDirId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + public returnSpace_call(long userId, long returnedBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { super(client, protocolFactory, transport, resultHandler, false); this.userId = userId; - this.storageDirId = storageDirId; this.returnedBytes = returnedBytes; } @@ -820,7 +694,6 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("returnSpace", org.apache.thrift.protocol.TMessageType.CALL, 0)); returnSpace_args args = new returnSpace_args(); args.setUserId(userId); - args.setStorageDirId(storageDirId); args.setReturnedBytes(returnedBytes); args.write(prot); prot.writeMessageEnd(); @@ -861,51 +734,13 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public ClientLocationInfo getResult() throws OutOfSpaceException, org.apache.thrift.TException { - if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_requestSpace(); - } - } - - public void requestSpaceInPlace(long userId, long storageDirId, long requestBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { - checkReady(); - requestSpaceInPlace_call method_call = new requestSpaceInPlace_call(userId, storageDirId, requestBytes, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class requestSpaceInPlace_call extends org.apache.thrift.async.TAsyncMethodCall { - private long userId; - private long storageDirId; - private long requestBytes; - public requestSpaceInPlace_call(long userId, long storageDirId, long requestBytes, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.userId = userId; - this.storageDirId = storageDirId; - this.requestBytes = requestBytes; - } - - public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { - prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("requestSpaceInPlace", org.apache.thrift.protocol.TMessageType.CALL, 0)); - requestSpaceInPlace_args args = new requestSpaceInPlace_args(); - args.setUserId(userId); - args.setStorageDirId(storageDirId); - args.setRequestBytes(requestBytes); - args.write(prot); - prot.writeMessageEnd(); - } - public boolean getResult() throws org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_requestSpaceInPlace(); + return (new Client(prot)).recv_requestSpace(); } } @@ -934,13 +769,13 @@ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apa prot.writeMessageEnd(); } - public boolean getResult() throws org.apache.thrift.TException { + public void getResult() throws org.apache.thrift.TException { if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { throw new IllegalStateException("Method call not finished!"); } org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_unlockBlock(); + (new Client(prot)).recv_unlockBlock(); } } @@ -993,14 +828,12 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { - public getLocalBlockLocation() { - super("getLocalBlockLocation"); + public static class getDataFolder extends org.apache.thrift.ProcessFunction { + public getDataFolder() { + super("getDataFolder"); } - public getLocalBlockLocation_args getEmptyArgsInstance() { - return new getLocalBlockLocation_args(); + public getDataFolder_args getEmptyArgsInstance() { + return new getDataFolder_args(); } protected boolean isOneway() { return false; } - public getLocalBlockLocation_result getResult(I iface, getLocalBlockLocation_args args) throws org.apache.thrift.TException { - getLocalBlockLocation_result result = new getLocalBlockLocation_result(); - try { - result.success = iface.getLocalBlockLocation(args.blockId); - } catch (FileDoesNotExistException eP) { - result.eP = eP; - } + public getDataFolder_result getResult(I iface, getDataFolder_args args) throws org.apache.thrift.TException { + getDataFolder_result result = new getDataFolder_result(); + result.success = iface.getDataFolder(); return result; } } - public static class getUserLocalTempFolder extends org.apache.thrift.ProcessFunction { - public getUserLocalTempFolder() { - super("getUserLocalTempFolder"); + public static class getUserTempFolder extends org.apache.thrift.ProcessFunction { + public getUserTempFolder() { + super("getUserTempFolder"); } - public getUserLocalTempFolder_args getEmptyArgsInstance() { - return new getUserLocalTempFolder_args(); + public getUserTempFolder_args getEmptyArgsInstance() { + return new getUserTempFolder_args(); } protected boolean isOneway() { return false; } - public getUserLocalTempFolder_result getResult(I iface, getUserLocalTempFolder_args args) throws org.apache.thrift.TException { - getUserLocalTempFolder_result result = new getUserLocalTempFolder_result(); - result.success = iface.getUserLocalTempFolder(args.userId, args.storageDirId); + public getUserTempFolder_result getResult(I iface, getUserTempFolder_args args) throws org.apache.thrift.TException { + getUserTempFolder_result result = new getUserTempFolder_result(); + result.success = iface.getUserTempFolder(args.userId); return result; } } @@ -1188,32 +1017,7 @@ protected boolean isOneway() { public lockBlock_result getResult(I iface, lockBlock_args args) throws org.apache.thrift.TException { lockBlock_result result = new lockBlock_result(); - try { - result.success = iface.lockBlock(args.blockId, args.userId); - } catch (FileDoesNotExistException eP) { - result.eP = eP; - } - return result; - } - } - - public static class promoteBlock extends org.apache.thrift.ProcessFunction { - public promoteBlock() { - super("promoteBlock"); - } - - public promoteBlock_args getEmptyArgsInstance() { - return new promoteBlock_args(); - } - - protected boolean isOneway() { - return false; - } - - public promoteBlock_result getResult(I iface, promoteBlock_args args) throws org.apache.thrift.TException { - promoteBlock_result result = new promoteBlock_result(); - result.success = iface.promoteBlock(args.userId, args.blockId); - result.setSuccessIsSet(true); + iface.lockBlock(args.blockId, args.userId); return result; } } @@ -1233,7 +1037,7 @@ protected boolean isOneway() { public returnSpace_result getResult(I iface, returnSpace_args args) throws org.apache.thrift.TException { returnSpace_result result = new returnSpace_result(); - iface.returnSpace(args.userId, args.storageDirId, args.returnedBytes); + iface.returnSpace(args.userId, args.returnedBytes); return result; } } @@ -1253,31 +1057,7 @@ protected boolean isOneway() { public requestSpace_result getResult(I iface, requestSpace_args args) throws org.apache.thrift.TException { requestSpace_result result = new requestSpace_result(); - try { - result.success = iface.requestSpace(args.userId, args.requestBytes); - } catch (OutOfSpaceException eP) { - result.eP = eP; - } - return result; - } - } - - public static class requestSpaceInPlace extends org.apache.thrift.ProcessFunction { - public requestSpaceInPlace() { - super("requestSpaceInPlace"); - } - - public requestSpaceInPlace_args getEmptyArgsInstance() { - return new requestSpaceInPlace_args(); - } - - protected boolean isOneway() { - return false; - } - - public requestSpaceInPlace_result getResult(I iface, requestSpaceInPlace_args args) throws org.apache.thrift.TException { - requestSpaceInPlace_result result = new requestSpaceInPlace_result(); - result.success = iface.requestSpaceInPlace(args.userId, args.storageDirId, args.requestBytes); + result.success = iface.requestSpace(args.userId, args.requestBytes); result.setSuccessIsSet(true); return result; } @@ -1298,8 +1078,7 @@ protected boolean isOneway() { public unlockBlock_result getResult(I iface, unlockBlock_args args) throws org.apache.thrift.TException { unlockBlock_result result = new unlockBlock_result(); - result.success = iface.unlockBlock(args.blockId, args.userId); - result.setSuccessIsSet(true); + iface.unlockBlock(args.blockId, args.userId); return result; } } @@ -1341,14 +1120,12 @@ protected AsyncProcessor(I iface, Map resultHandler) throws TException { - iface.accessBlock(args.storageDirId, args.blockId,resultHandler); + iface.accessBlock(args.blockId,resultHandler); } } @@ -1595,24 +1372,24 @@ protected boolean isOneway() { } public void start(I iface, cacheBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.cacheBlock(args.userId, args.storageDirId, args.blockId,resultHandler); + iface.cacheBlock(args.userId, args.blockId,resultHandler); } } - public static class getLocalBlockLocation extends org.apache.thrift.AsyncProcessFunction { - public getLocalBlockLocation() { - super("getLocalBlockLocation"); + public static class getDataFolder extends org.apache.thrift.AsyncProcessFunction { + public getDataFolder() { + super("getDataFolder"); } - public getLocalBlockLocation_args getEmptyArgsInstance() { - return new getLocalBlockLocation_args(); + public getDataFolder_args getEmptyArgsInstance() { + return new getDataFolder_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(ClientLocationInfo o) { - getLocalBlockLocation_result result = new getLocalBlockLocation_result(); + return new AsyncMethodCallback() { + public void onComplete(String o) { + getDataFolder_result result = new getDataFolder_result(); result.success = o; try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); @@ -1625,13 +1402,7 @@ public void onComplete(ClientLocationInfo o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - getLocalBlockLocation_result result = new getLocalBlockLocation_result(); - if (e instanceof FileDoesNotExistException) { - result.eP = (FileDoesNotExistException) e; - result.setEPIsSet(true); - msg = result; - } - else + getDataFolder_result result = new getDataFolder_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1651,25 +1422,25 @@ protected boolean isOneway() { return false; } - public void start(I iface, getLocalBlockLocation_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.getLocalBlockLocation(args.blockId,resultHandler); + public void start(I iface, getDataFolder_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getDataFolder(resultHandler); } } - public static class getUserLocalTempFolder extends org.apache.thrift.AsyncProcessFunction { - public getUserLocalTempFolder() { - super("getUserLocalTempFolder"); + public static class getUserTempFolder extends org.apache.thrift.AsyncProcessFunction { + public getUserTempFolder() { + super("getUserTempFolder"); } - public getUserLocalTempFolder_args getEmptyArgsInstance() { - return new getUserLocalTempFolder_args(); + public getUserTempFolder_args getEmptyArgsInstance() { + return new getUserTempFolder_args(); } public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; return new AsyncMethodCallback() { public void onComplete(String o) { - getUserLocalTempFolder_result result = new getUserLocalTempFolder_result(); + getUserTempFolder_result result = new getUserTempFolder_result(); result.success = o; try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); @@ -1682,7 +1453,7 @@ public void onComplete(String o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - getUserLocalTempFolder_result result = new getUserLocalTempFolder_result(); + getUserTempFolder_result result = new getUserTempFolder_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1702,8 +1473,8 @@ protected boolean isOneway() { return false; } - public void start(I iface, getUserLocalTempFolder_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.getUserLocalTempFolder(args.userId, args.storageDirId,resultHandler); + public void start(I iface, getUserTempFolder_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getUserTempFolder(args.userId,resultHandler); } } @@ -1758,7 +1529,7 @@ public void start(I iface, getUserUfsTempFolder_args args, org.apache.thrift.asy } } - public static class lockBlock extends org.apache.thrift.AsyncProcessFunction { + public static class lockBlock extends org.apache.thrift.AsyncProcessFunction { public lockBlock() { super("lockBlock"); } @@ -1767,12 +1538,11 @@ public lockBlock_args getEmptyArgsInstance() { return new lockBlock_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(ClientLocationInfo o) { + return new AsyncMethodCallback() { + public void onComplete(Void o) { lockBlock_result result = new lockBlock_result(); - result.success = o; try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); return; @@ -1785,12 +1555,6 @@ public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; lockBlock_result result = new lockBlock_result(); - if (e instanceof FileDoesNotExistException) { - result.eP = (FileDoesNotExistException) e; - result.setEPIsSet(true); - msg = result; - } - else { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1810,27 +1574,25 @@ protected boolean isOneway() { return false; } - public void start(I iface, lockBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + public void start(I iface, lockBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { iface.lockBlock(args.blockId, args.userId,resultHandler); } } - public static class promoteBlock extends org.apache.thrift.AsyncProcessFunction { - public promoteBlock() { - super("promoteBlock"); + public static class returnSpace extends org.apache.thrift.AsyncProcessFunction { + public returnSpace() { + super("returnSpace"); } - public promoteBlock_args getEmptyArgsInstance() { - return new promoteBlock_args(); + public returnSpace_args getEmptyArgsInstance() { + return new returnSpace_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(Boolean o) { - promoteBlock_result result = new promoteBlock_result(); - result.success = o; - result.setSuccessIsSet(true); + return new AsyncMethodCallback() { + public void onComplete(Void o) { + returnSpace_result result = new returnSpace_result(); try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); return; @@ -1842,7 +1604,7 @@ public void onComplete(Boolean o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - promoteBlock_result result = new promoteBlock_result(); + returnSpace_result result = new returnSpace_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1862,25 +1624,27 @@ protected boolean isOneway() { return false; } - public void start(I iface, promoteBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.promoteBlock(args.userId, args.blockId,resultHandler); + public void start(I iface, returnSpace_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.returnSpace(args.userId, args.returnedBytes,resultHandler); } } - public static class returnSpace extends org.apache.thrift.AsyncProcessFunction { - public returnSpace() { - super("returnSpace"); + public static class requestSpace extends org.apache.thrift.AsyncProcessFunction { + public requestSpace() { + super("requestSpace"); } - public returnSpace_args getEmptyArgsInstance() { - return new returnSpace_args(); + public requestSpace_args getEmptyArgsInstance() { + return new requestSpace_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(Void o) { - returnSpace_result result = new returnSpace_result(); + return new AsyncMethodCallback() { + public void onComplete(Boolean o) { + requestSpace_result result = new requestSpace_result(); + result.success = o; + result.setSuccessIsSet(true); try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); return; @@ -1892,7 +1656,7 @@ public void onComplete(Void o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - returnSpace_result result = new returnSpace_result(); + requestSpace_result result = new requestSpace_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1912,26 +1676,25 @@ protected boolean isOneway() { return false; } - public void start(I iface, returnSpace_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.returnSpace(args.userId, args.storageDirId, args.returnedBytes,resultHandler); + public void start(I iface, requestSpace_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.requestSpace(args.userId, args.requestBytes,resultHandler); } } - public static class requestSpace extends org.apache.thrift.AsyncProcessFunction { - public requestSpace() { - super("requestSpace"); + public static class unlockBlock extends org.apache.thrift.AsyncProcessFunction { + public unlockBlock() { + super("unlockBlock"); } - public requestSpace_args getEmptyArgsInstance() { - return new requestSpace_args(); + public unlockBlock_args getEmptyArgsInstance() { + return new unlockBlock_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(ClientLocationInfo o) { - requestSpace_result result = new requestSpace_result(); - result.success = o; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + unlockBlock_result result = new unlockBlock_result(); try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); return; @@ -1943,13 +1706,7 @@ public void onComplete(ClientLocationInfo o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - requestSpace_result result = new requestSpace_result(); - if (e instanceof OutOfSpaceException) { - result.eP = (OutOfSpaceException) e; - result.setEPIsSet(true); - msg = result; - } - else + unlockBlock_result result = new unlockBlock_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -1969,27 +1726,25 @@ protected boolean isOneway() { return false; } - public void start(I iface, requestSpace_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.requestSpace(args.userId, args.requestBytes,resultHandler); + public void start(I iface, unlockBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.unlockBlock(args.blockId, args.userId,resultHandler); } } - public static class requestSpaceInPlace extends org.apache.thrift.AsyncProcessFunction { - public requestSpaceInPlace() { - super("requestSpaceInPlace"); + public static class userHeartbeat extends org.apache.thrift.AsyncProcessFunction { + public userHeartbeat() { + super("userHeartbeat"); } - public requestSpaceInPlace_args getEmptyArgsInstance() { - return new requestSpaceInPlace_args(); + public userHeartbeat_args getEmptyArgsInstance() { + return new userHeartbeat_args(); } - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(Boolean o) { - requestSpaceInPlace_result result = new requestSpaceInPlace_result(); - result.success = o; - result.setSuccessIsSet(true); + return new AsyncMethodCallback() { + public void onComplete(Void o) { + userHeartbeat_result result = new userHeartbeat_result(); try { fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); return; @@ -2001,7 +1756,7 @@ public void onComplete(Boolean o) { public void onError(Exception e) { byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; org.apache.thrift.TBase msg; - requestSpaceInPlace_result result = new requestSpaceInPlace_result(); + userHeartbeat_result result = new userHeartbeat_result(); { msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); @@ -2021,120 +1776,17 @@ protected boolean isOneway() { return false; } - public void start(I iface, requestSpaceInPlace_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.requestSpaceInPlace(args.userId, args.storageDirId, args.requestBytes,resultHandler); + public void start(I iface, userHeartbeat_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.userHeartbeat(args.userId,resultHandler); } } - public static class unlockBlock extends org.apache.thrift.AsyncProcessFunction { - public unlockBlock() { - super("unlockBlock"); - } - - public unlockBlock_args getEmptyArgsInstance() { - return new unlockBlock_args(); - } - - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { - final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(Boolean o) { - unlockBlock_result result = new unlockBlock_result(); - result.success = o; - result.setSuccessIsSet(true); - try { - fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); - return; - } catch (Exception e) { - LOGGER.error("Exception writing to internal frame buffer", e); - } - fb.close(); - } - public void onError(Exception e) { - byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; - org.apache.thrift.TBase msg; - unlockBlock_result result = new unlockBlock_result(); - { - msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; - msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); - } - try { - fcall.sendResponse(fb,msg,msgType,seqid); - return; - } catch (Exception ex) { - LOGGER.error("Exception writing to internal frame buffer", ex); - } - fb.close(); - } - }; - } - - protected boolean isOneway() { - return false; - } - - public void start(I iface, unlockBlock_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.unlockBlock(args.blockId, args.userId,resultHandler); - } - } - - public static class userHeartbeat extends org.apache.thrift.AsyncProcessFunction { - public userHeartbeat() { - super("userHeartbeat"); - } - - public userHeartbeat_args getEmptyArgsInstance() { - return new userHeartbeat_args(); - } - - public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { - final org.apache.thrift.AsyncProcessFunction fcall = this; - return new AsyncMethodCallback() { - public void onComplete(Void o) { - userHeartbeat_result result = new userHeartbeat_result(); - try { - fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); - return; - } catch (Exception e) { - LOGGER.error("Exception writing to internal frame buffer", e); - } - fb.close(); - } - public void onError(Exception e) { - byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; - org.apache.thrift.TBase msg; - userHeartbeat_result result = new userHeartbeat_result(); - { - msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; - msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); - } - try { - fcall.sendResponse(fb,msg,msgType,seqid); - return; - } catch (Exception ex) { - LOGGER.error("Exception writing to internal frame buffer", ex); - } - fb.close(); - } - }; - } - - protected boolean isOneway() { - return false; - } - - public void start(I iface, userHeartbeat_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { - iface.userHeartbeat(args.userId,resultHandler); - } - } - - } + } public static class accessBlock_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("accessBlock_args"); - private static final org.apache.thrift.protocol.TField STORAGE_DIR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storageDirId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField BLOCK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("blockId", org.apache.thrift.protocol.TType.I64, (short)2); + private static final org.apache.thrift.protocol.TField BLOCK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("blockId", org.apache.thrift.protocol.TType.I64, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -2142,13 +1794,11 @@ public static class accessBlock_args implements org.apache.thrift.TBase byName = new HashMap(); @@ -2163,9 +1813,7 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 2: // BLOCK_ID + case 1: // BLOCK_ID return BLOCK_ID; default: return null; @@ -2207,14 +1855,11 @@ public String getFieldName() { } // isset id assignments - private static final int __STORAGEDIRID_ISSET_ID = 0; - private static final int __BLOCKID_ISSET_ID = 1; + private static final int __BLOCKID_ISSET_ID = 0; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.BLOCK_ID, new org.apache.thrift.meta_data.FieldMetaData("blockId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); @@ -2225,12 +1870,9 @@ public accessBlock_args() { } public accessBlock_args( - long storageDirId, long blockId) { this(); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); this.blockId = blockId; setBlockIdIsSet(true); } @@ -2240,7 +1882,6 @@ public accessBlock_args( */ public accessBlock_args(accessBlock_args other) { __isset_bitfield = other.__isset_bitfield; - this.storageDirId = other.storageDirId; this.blockId = other.blockId; } @@ -2250,35 +1891,10 @@ public accessBlock_args deepCopy() { @Override public void clear() { - setStorageDirIdIsSet(false); - this.storageDirId = 0; setBlockIdIsSet(false); this.blockId = 0; } - public long getStorageDirId() { - return this.storageDirId; - } - - public accessBlock_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - public long getBlockId() { return this.blockId; } @@ -2304,14 +1920,6 @@ public void setBlockIdIsSet(boolean value) { public void setFieldValue(_Fields field, Object value) { switch (field) { - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - case BLOCK_ID: if (value == null) { unsetBlockId(); @@ -2325,9 +1933,6 @@ public void setFieldValue(_Fields field, Object value) { public Object getFieldValue(_Fields field) { switch (field) { - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - case BLOCK_ID: return Long.valueOf(getBlockId()); @@ -2342,8 +1947,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case STORAGE_DIR_ID: - return isSetStorageDirId(); case BLOCK_ID: return isSetBlockId(); } @@ -2363,15 +1966,6 @@ public boolean equals(accessBlock_args that) { if (that == null) return false; - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - boolean this_present_blockId = true; boolean that_present_blockId = true; if (this_present_blockId || that_present_blockId) { @@ -2397,16 +1991,6 @@ public int compareTo(accessBlock_args other) { int lastComparison = 0; - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(isSetBlockId()).compareTo(other.isSetBlockId()); if (lastComparison != 0) { return lastComparison; @@ -2437,10 +2021,6 @@ public String toString() { StringBuilder sb = new StringBuilder("accessBlock_args("); boolean first = true; - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); sb.append("blockId:"); sb.append(this.blockId); first = false; @@ -2489,15 +2069,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, accessBlock_args st break; } switch (schemeField.id) { - case 1: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // BLOCK_ID + case 1: // BLOCK_ID if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); @@ -2520,9 +2092,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, accessBlock_args s struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); oprot.writeFieldBegin(BLOCK_ID_FIELD_DESC); oprot.writeI64(struct.blockId); oprot.writeFieldEnd(); @@ -2544,16 +2113,10 @@ private static class accessBlock_argsTupleScheme extends TupleScheme, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -4752,14 +4310,12 @@ public static class cacheBlock_args implements org.apache.thrift.TBase byName = new HashMap(); @@ -4776,9 +4332,7 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // USER_ID return USER_ID; - case 2: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 3: // BLOCK_ID + case 2: // BLOCK_ID return BLOCK_ID; default: return null; @@ -4821,16 +4375,13 @@ public String getFieldName() { // isset id assignments private static final int __USERID_ISSET_ID = 0; - private static final int __STORAGEDIRID_ISSET_ID = 1; - private static final int __BLOCKID_ISSET_ID = 2; + private static final int __BLOCKID_ISSET_ID = 1; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.BLOCK_ID, new org.apache.thrift.meta_data.FieldMetaData("blockId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); @@ -4842,14 +4393,11 @@ public cacheBlock_args() { public cacheBlock_args( long userId, - long storageDirId, long blockId) { this(); this.userId = userId; setUserIdIsSet(true); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); this.blockId = blockId; setBlockIdIsSet(true); } @@ -4860,7 +4408,6 @@ public cacheBlock_args( public cacheBlock_args(cacheBlock_args other) { __isset_bitfield = other.__isset_bitfield; this.userId = other.userId; - this.storageDirId = other.storageDirId; this.blockId = other.blockId; } @@ -4872,8 +4419,6 @@ public cacheBlock_args deepCopy() { public void clear() { setUserIdIsSet(false); this.userId = 0; - setStorageDirIdIsSet(false); - this.storageDirId = 0; setBlockIdIsSet(false); this.blockId = 0; } @@ -4901,29 +4446,6 @@ public void setUserIdIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); } - public long getStorageDirId() { - return this.storageDirId; - } - - public cacheBlock_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - public long getBlockId() { return this.blockId; } @@ -4957,14 +4479,6 @@ public void setFieldValue(_Fields field, Object value) { } break; - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - case BLOCK_ID: if (value == null) { unsetBlockId(); @@ -4981,9 +4495,6 @@ public Object getFieldValue(_Fields field) { case USER_ID: return Long.valueOf(getUserId()); - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - case BLOCK_ID: return Long.valueOf(getBlockId()); @@ -5000,8 +4511,6 @@ public boolean isSet(_Fields field) { switch (field) { case USER_ID: return isSetUserId(); - case STORAGE_DIR_ID: - return isSetStorageDirId(); case BLOCK_ID: return isSetBlockId(); } @@ -5030,15 +4539,6 @@ public boolean equals(cacheBlock_args that) { return false; } - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - boolean this_present_blockId = true; boolean that_present_blockId = true; if (this_present_blockId || that_present_blockId) { @@ -5074,16 +4574,6 @@ public int compareTo(cacheBlock_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(isSetBlockId()).compareTo(other.isSetBlockId()); if (lastComparison != 0) { return lastComparison; @@ -5118,10 +4608,6 @@ public String toString() { sb.append(this.userId); first = false; if (!first) sb.append(", "); - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); sb.append("blockId:"); sb.append(this.blockId); first = false; @@ -5178,15 +4664,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, cacheBlock_args str org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // BLOCK_ID + case 2: // BLOCK_ID if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); @@ -5212,9 +4690,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, cacheBlock_args st oprot.writeFieldBegin(USER_ID_FIELD_DESC); oprot.writeI64(struct.userId); oprot.writeFieldEnd(); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); oprot.writeFieldBegin(BLOCK_ID_FIELD_DESC); oprot.writeI64(struct.blockId); oprot.writeFieldEnd(); @@ -5239,19 +4714,13 @@ public void write(org.apache.thrift.protocol.TProtocol prot, cacheBlock_args str if (struct.isSetUserId()) { optionals.set(0); } - if (struct.isSetStorageDirId()) { - optionals.set(1); - } if (struct.isSetBlockId()) { - optionals.set(2); + optionals.set(1); } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); if (struct.isSetUserId()) { oprot.writeI64(struct.userId); } - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } if (struct.isSetBlockId()) { oprot.writeI64(struct.blockId); } @@ -5260,16 +4729,12 @@ public void write(org.apache.thrift.protocol.TProtocol prot, cacheBlock_args str @Override public void read(org.apache.thrift.protocol.TProtocol prot, cacheBlock_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { struct.userId = iprot.readI64(); struct.setUserIdIsSet(true); } if (incoming.get(1)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } - if (incoming.get(2)) { struct.blockId = iprot.readI64(); struct.setBlockIdIsSet(true); } @@ -5838,22 +5303,20 @@ public void read(org.apache.thrift.protocol.TProtocol prot, cacheBlock_result st } - public static class getLocalBlockLocation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLocalBlockLocation_args"); + public static class getDataFolder_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getDataFolder_args"); - private static final org.apache.thrift.protocol.TField BLOCK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("blockId", org.apache.thrift.protocol.TType.I64, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getLocalBlockLocation_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getLocalBlockLocation_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getDataFolder_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getDataFolder_argsTupleSchemeFactory()); } - public long blockId; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - BLOCK_ID((short)1, "blockId"); +; private static final Map byName = new HashMap(); @@ -5868,8 +5331,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 1: // BLOCK_ID - return BLOCK_ID; default: return null; } @@ -5908,128 +5369,65 @@ public String getFieldName() { return _fieldName; } } - - // isset id assignments - private static final int __BLOCKID_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BLOCK_ID, new org.apache.thrift.meta_data.FieldMetaData("blockId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLocalBlockLocation_args.class, metaDataMap); - } - - public getLocalBlockLocation_args() { + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getDataFolder_args.class, metaDataMap); } - public getLocalBlockLocation_args( - long blockId) - { - this(); - this.blockId = blockId; - setBlockIdIsSet(true); + public getDataFolder_args() { } /** * Performs a deep copy on other. */ - public getLocalBlockLocation_args(getLocalBlockLocation_args other) { - __isset_bitfield = other.__isset_bitfield; - this.blockId = other.blockId; + public getDataFolder_args(getDataFolder_args other) { } - public getLocalBlockLocation_args deepCopy() { - return new getLocalBlockLocation_args(this); + public getDataFolder_args deepCopy() { + return new getDataFolder_args(this); } @Override public void clear() { - setBlockIdIsSet(false); - this.blockId = 0; } - public long getBlockId() { - return this.blockId; + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } } - public getLocalBlockLocation_args setBlockId(long blockId) { - this.blockId = blockId; - setBlockIdIsSet(true); - return this; + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); } - public void unsetBlockId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BLOCKID_ISSET_ID); - } + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } - /** Returns true if field blockId is set (has been assigned a value) and false otherwise */ - public boolean isSetBlockId() { - return EncodingUtils.testBit(__isset_bitfield, __BLOCKID_ISSET_ID); - } - - public void setBlockIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BLOCKID_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case BLOCK_ID: - if (value == null) { - unsetBlockId(); - } else { - setBlockId((Long)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case BLOCK_ID: - return Long.valueOf(getBlockId()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case BLOCK_ID: - return isSetBlockId(); - } - throw new IllegalStateException(); + switch (field) { + } + throw new IllegalStateException(); } @Override public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getLocalBlockLocation_args) - return this.equals((getLocalBlockLocation_args)that); + if (that instanceof getDataFolder_args) + return this.equals((getDataFolder_args)that); return false; } - public boolean equals(getLocalBlockLocation_args that) { + public boolean equals(getDataFolder_args that) { if (that == null) return false; - boolean this_present_blockId = true; - boolean that_present_blockId = true; - if (this_present_blockId || that_present_blockId) { - if (!(this_present_blockId && that_present_blockId)) - return false; - if (this.blockId != that.blockId) - return false; - } - return true; } @@ -6039,23 +5437,13 @@ public int hashCode() { } @Override - public int compareTo(getLocalBlockLocation_args other) { + public int compareTo(getDataFolder_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(isSetBlockId()).compareTo(other.isSetBlockId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetBlockId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.blockId, other.blockId); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -6073,12 +5461,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getLocalBlockLocation_args("); + StringBuilder sb = new StringBuilder("getDataFolder_args("); boolean first = true; - sb.append("blockId:"); - sb.append(this.blockId); - first = false; sb.append(")"); return sb.toString(); } @@ -6098,23 +5483,21 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); } } - private static class getLocalBlockLocation_argsStandardSchemeFactory implements SchemeFactory { - public getLocalBlockLocation_argsStandardScheme getScheme() { - return new getLocalBlockLocation_argsStandardScheme(); + private static class getDataFolder_argsStandardSchemeFactory implements SchemeFactory { + public getDataFolder_argsStandardScheme getScheme() { + return new getDataFolder_argsStandardScheme(); } } - private static class getLocalBlockLocation_argsStandardScheme extends StandardScheme { + private static class getDataFolder_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocation_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getDataFolder_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -6124,14 +5507,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocati break; } switch (schemeField.id) { - case 1: // BLOCK_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.blockId = iprot.readI64(); - struct.setBlockIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -6143,72 +5518,53 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocati struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getLocalBlockLocation_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getDataFolder_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(BLOCK_ID_FIELD_DESC); - oprot.writeI64(struct.blockId); - oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getLocalBlockLocation_argsTupleSchemeFactory implements SchemeFactory { - public getLocalBlockLocation_argsTupleScheme getScheme() { - return new getLocalBlockLocation_argsTupleScheme(); + private static class getDataFolder_argsTupleSchemeFactory implements SchemeFactory { + public getDataFolder_argsTupleScheme getScheme() { + return new getDataFolder_argsTupleScheme(); } } - private static class getLocalBlockLocation_argsTupleScheme extends TupleScheme { + private static class getDataFolder_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getLocalBlockLocation_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getDataFolder_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetBlockId()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetBlockId()) { - oprot.writeI64(struct.blockId); - } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getLocalBlockLocation_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getDataFolder_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.blockId = iprot.readI64(); - struct.setBlockIdIsSet(true); - } } } } - public static class getLocalBlockLocation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLocalBlockLocation_result"); + public static class getDataFolder_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getDataFolder_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift.protocol.TField E_P_FIELD_DESC = new org.apache.thrift.protocol.TField("eP", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getLocalBlockLocation_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getLocalBlockLocation_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getDataFolder_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getDataFolder_resultTupleSchemeFactory()); } - public ClientLocationInfo success; // required - public FileDoesNotExistException eP; // required + public String success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"), - E_P((short)1, "eP"); + SUCCESS((short)0, "success"); private static final Map byName = new HashMap(); @@ -6225,8 +5581,6 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 0: // SUCCESS return SUCCESS; - case 1: // E_P - return E_P; default: return null; } @@ -6271,52 +5625,44 @@ public String getFieldName() { static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientLocationInfo.class))); - tmpMap.put(_Fields.E_P, new org.apache.thrift.meta_data.FieldMetaData("eP", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLocalBlockLocation_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getDataFolder_result.class, metaDataMap); } - public getLocalBlockLocation_result() { + public getDataFolder_result() { } - public getLocalBlockLocation_result( - ClientLocationInfo success, - FileDoesNotExistException eP) + public getDataFolder_result( + String success) { this(); this.success = success; - this.eP = eP; } /** * Performs a deep copy on other. */ - public getLocalBlockLocation_result(getLocalBlockLocation_result other) { + public getDataFolder_result(getDataFolder_result other) { if (other.isSetSuccess()) { - this.success = new ClientLocationInfo(other.success); - } - if (other.isSetEP()) { - this.eP = new FileDoesNotExistException(other.eP); + this.success = other.success; } } - public getLocalBlockLocation_result deepCopy() { - return new getLocalBlockLocation_result(this); + public getDataFolder_result deepCopy() { + return new getDataFolder_result(this); } @Override public void clear() { this.success = null; - this.eP = null; } - public ClientLocationInfo getSuccess() { + public String getSuccess() { return this.success; } - public getLocalBlockLocation_result setSuccess(ClientLocationInfo success) { + public getDataFolder_result setSuccess(String success) { this.success = success; return this; } @@ -6336,45 +5682,13 @@ public void setSuccessIsSet(boolean value) { } } - public FileDoesNotExistException getEP() { - return this.eP; - } - - public getLocalBlockLocation_result setEP(FileDoesNotExistException eP) { - this.eP = eP; - return this; - } - - public void unsetEP() { - this.eP = null; - } - - /** Returns true if field eP is set (has been assigned a value) and false otherwise */ - public boolean isSetEP() { - return this.eP != null; - } - - public void setEPIsSet(boolean value) { - if (!value) { - this.eP = null; - } - } - public void setFieldValue(_Fields field, Object value) { switch (field) { case SUCCESS: if (value == null) { unsetSuccess(); } else { - setSuccess((ClientLocationInfo)value); - } - break; - - case E_P: - if (value == null) { - unsetEP(); - } else { - setEP((FileDoesNotExistException)value); + setSuccess((String)value); } break; @@ -6386,9 +5700,6 @@ public Object getFieldValue(_Fields field) { case SUCCESS: return getSuccess(); - case E_P: - return getEP(); - } throw new IllegalStateException(); } @@ -6402,8 +5713,6 @@ public boolean isSet(_Fields field) { switch (field) { case SUCCESS: return isSetSuccess(); - case E_P: - return isSetEP(); } throw new IllegalStateException(); } @@ -6412,12 +5721,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getLocalBlockLocation_result) - return this.equals((getLocalBlockLocation_result)that); + if (that instanceof getDataFolder_result) + return this.equals((getDataFolder_result)that); return false; } - public boolean equals(getLocalBlockLocation_result that) { + public boolean equals(getDataFolder_result that) { if (that == null) return false; @@ -6430,15 +5739,6 @@ public boolean equals(getLocalBlockLocation_result that) { return false; } - boolean this_present_eP = true && this.isSetEP(); - boolean that_present_eP = true && that.isSetEP(); - if (this_present_eP || that_present_eP) { - if (!(this_present_eP && that_present_eP)) - return false; - if (!this.eP.equals(that.eP)) - return false; - } - return true; } @@ -6448,7 +5748,7 @@ public int hashCode() { } @Override - public int compareTo(getLocalBlockLocation_result other) { + public int compareTo(getDataFolder_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -6465,16 +5765,6 @@ public int compareTo(getLocalBlockLocation_result other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetEP()).compareTo(other.isSetEP()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetEP()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eP, other.eP); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -6492,7 +5782,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getLocalBlockLocation_result("); + StringBuilder sb = new StringBuilder("getDataFolder_result("); boolean first = true; sb.append("success:"); @@ -6502,14 +5792,6 @@ public String toString() { sb.append(this.success); } first = false; - if (!first) sb.append(", "); - sb.append("eP:"); - if (this.eP == null) { - sb.append("null"); - } else { - sb.append(this.eP); - } - first = false; sb.append(")"); return sb.toString(); } @@ -6517,9 +5799,6 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity - if (success != null) { - success.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -6538,15 +5817,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getLocalBlockLocation_resultStandardSchemeFactory implements SchemeFactory { - public getLocalBlockLocation_resultStandardScheme getScheme() { - return new getLocalBlockLocation_resultStandardScheme(); + private static class getDataFolder_resultStandardSchemeFactory implements SchemeFactory { + public getDataFolder_resultStandardScheme getScheme() { + return new getDataFolder_resultStandardScheme(); } } - private static class getLocalBlockLocation_resultStandardScheme extends StandardScheme { + private static class getDataFolder_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocation_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getDataFolder_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -6557,23 +5836,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocati } switch (schemeField.id) { case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new ClientLocationInfo(); - struct.success.read(iprot); + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); struct.setSuccessIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 1: // E_P - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.eP = new FileDoesNotExistException(); - struct.eP.read(iprot); - struct.setEPIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -6585,18 +5854,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getLocalBlockLocati struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getLocalBlockLocation_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getDataFolder_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); if (struct.success != null) { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.eP != null) { - oprot.writeFieldBegin(E_P_FIELD_DESC); - struct.eP.write(oprot); + oprot.writeString(struct.success); oprot.writeFieldEnd(); } oprot.writeFieldStop(); @@ -6605,71 +5869,56 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getLocalBlockLocat } - private static class getLocalBlockLocation_resultTupleSchemeFactory implements SchemeFactory { - public getLocalBlockLocation_resultTupleScheme getScheme() { - return new getLocalBlockLocation_resultTupleScheme(); + private static class getDataFolder_resultTupleSchemeFactory implements SchemeFactory { + public getDataFolder_resultTupleScheme getScheme() { + return new getDataFolder_resultTupleScheme(); } } - private static class getLocalBlockLocation_resultTupleScheme extends TupleScheme { + private static class getDataFolder_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getLocalBlockLocation_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getDataFolder_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetSuccess()) { optionals.set(0); } - if (struct.isSetEP()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); + oprot.writeBitSet(optionals, 1); if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - if (struct.isSetEP()) { - struct.eP.write(oprot); + oprot.writeString(struct.success); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getLocalBlockLocation_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getDataFolder_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); + BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { - struct.success = new ClientLocationInfo(); - struct.success.read(iprot); + struct.success = iprot.readString(); struct.setSuccessIsSet(true); } - if (incoming.get(1)) { - struct.eP = new FileDoesNotExistException(); - struct.eP.read(iprot); - struct.setEPIsSet(true); - } } } } - public static class getUserLocalTempFolder_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserLocalTempFolder_args"); + public static class getUserTempFolder_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTempFolder_args"); private static final org.apache.thrift.protocol.TField USER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("userId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField STORAGE_DIR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storageDirId", org.apache.thrift.protocol.TType.I64, (short)2); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getUserLocalTempFolder_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getUserLocalTempFolder_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getUserTempFolder_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTempFolder_argsTupleSchemeFactory()); } public long userId; // required - public long storageDirId; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - USER_ID((short)1, "userId"), - STORAGE_DIR_ID((short)2, "storageDirId"); + USER_ID((short)1, "userId"); private static final Map byName = new HashMap(); @@ -6686,8 +5935,6 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // USER_ID return USER_ID; - case 2: // STORAGE_DIR_ID - return STORAGE_DIR_ID; default: return null; } @@ -6729,59 +5976,50 @@ public String getFieldName() { // isset id assignments private static final int __USERID_ISSET_ID = 0; - private static final int __STORAGEDIRID_ISSET_ID = 1; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserLocalTempFolder_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTempFolder_args.class, metaDataMap); } - public getUserLocalTempFolder_args() { + public getUserTempFolder_args() { } - public getUserLocalTempFolder_args( - long userId, - long storageDirId) + public getUserTempFolder_args( + long userId) { this(); this.userId = userId; setUserIdIsSet(true); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); } /** * Performs a deep copy on other. */ - public getUserLocalTempFolder_args(getUserLocalTempFolder_args other) { + public getUserTempFolder_args(getUserTempFolder_args other) { __isset_bitfield = other.__isset_bitfield; this.userId = other.userId; - this.storageDirId = other.storageDirId; } - public getUserLocalTempFolder_args deepCopy() { - return new getUserLocalTempFolder_args(this); + public getUserTempFolder_args deepCopy() { + return new getUserTempFolder_args(this); } @Override public void clear() { setUserIdIsSet(false); this.userId = 0; - setStorageDirIdIsSet(false); - this.storageDirId = 0; } public long getUserId() { return this.userId; } - public getUserLocalTempFolder_args setUserId(long userId) { + public getUserTempFolder_args setUserId(long userId) { this.userId = userId; setUserIdIsSet(true); return this; @@ -6800,29 +6038,6 @@ public void setUserIdIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); } - public long getStorageDirId() { - return this.storageDirId; - } - - public getUserLocalTempFolder_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - public void setFieldValue(_Fields field, Object value) { switch (field) { case USER_ID: @@ -6833,14 +6048,6 @@ public void setFieldValue(_Fields field, Object value) { } break; - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - } } @@ -6849,9 +6056,6 @@ public Object getFieldValue(_Fields field) { case USER_ID: return Long.valueOf(getUserId()); - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - } throw new IllegalStateException(); } @@ -6865,8 +6069,6 @@ public boolean isSet(_Fields field) { switch (field) { case USER_ID: return isSetUserId(); - case STORAGE_DIR_ID: - return isSetStorageDirId(); } throw new IllegalStateException(); } @@ -6875,12 +6077,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getUserLocalTempFolder_args) - return this.equals((getUserLocalTempFolder_args)that); + if (that instanceof getUserTempFolder_args) + return this.equals((getUserTempFolder_args)that); return false; } - public boolean equals(getUserLocalTempFolder_args that) { + public boolean equals(getUserTempFolder_args that) { if (that == null) return false; @@ -6893,15 +6095,6 @@ public boolean equals(getUserLocalTempFolder_args that) { return false; } - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - return true; } @@ -6911,7 +6104,7 @@ public int hashCode() { } @Override - public int compareTo(getUserLocalTempFolder_args other) { + public int compareTo(getUserTempFolder_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -6928,16 +6121,6 @@ public int compareTo(getUserLocalTempFolder_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -6955,16 +6138,12 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getUserLocalTempFolder_args("); + StringBuilder sb = new StringBuilder("getUserTempFolder_args("); boolean first = true; sb.append("userId:"); sb.append(this.userId); first = false; - if (!first) sb.append(", "); - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; sb.append(")"); return sb.toString(); } @@ -6992,15 +6171,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getUserLocalTempFolder_argsStandardSchemeFactory implements SchemeFactory { - public getUserLocalTempFolder_argsStandardScheme getScheme() { - return new getUserLocalTempFolder_argsStandardScheme(); + private static class getUserTempFolder_argsStandardSchemeFactory implements SchemeFactory { + public getUserTempFolder_argsStandardScheme getScheme() { + return new getUserTempFolder_argsStandardScheme(); } } - private static class getUserLocalTempFolder_argsStandardScheme extends StandardScheme { + private static class getUserTempFolder_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getUserLocalTempFolder_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTempFolder_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -7018,14 +6197,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserLocalTempFol org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -7037,75 +6208,62 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserLocalTempFol struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getUserLocalTempFolder_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTempFolder_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldBegin(USER_ID_FIELD_DESC); oprot.writeI64(struct.userId); oprot.writeFieldEnd(); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class getUserLocalTempFolder_argsTupleSchemeFactory implements SchemeFactory { - public getUserLocalTempFolder_argsTupleScheme getScheme() { - return new getUserLocalTempFolder_argsTupleScheme(); + private static class getUserTempFolder_argsTupleSchemeFactory implements SchemeFactory { + public getUserTempFolder_argsTupleScheme getScheme() { + return new getUserTempFolder_argsTupleScheme(); } } - private static class getUserLocalTempFolder_argsTupleScheme extends TupleScheme { + private static class getUserTempFolder_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getUserLocalTempFolder_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTempFolder_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetUserId()) { optionals.set(0); } - if (struct.isSetStorageDirId()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); + oprot.writeBitSet(optionals, 1); if (struct.isSetUserId()) { oprot.writeI64(struct.userId); } - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getUserLocalTempFolder_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTempFolder_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); + BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { struct.userId = iprot.readI64(); struct.setUserIdIsSet(true); } - if (incoming.get(1)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } } } } - public static class getUserLocalTempFolder_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserLocalTempFolder_result"); + public static class getUserTempFolder_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTempFolder_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new getUserLocalTempFolder_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new getUserLocalTempFolder_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new getUserTempFolder_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTempFolder_resultTupleSchemeFactory()); } public String success; // required @@ -7175,13 +6333,13 @@ public String getFieldName() { tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserLocalTempFolder_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTempFolder_result.class, metaDataMap); } - public getUserLocalTempFolder_result() { + public getUserTempFolder_result() { } - public getUserLocalTempFolder_result( + public getUserTempFolder_result( String success) { this(); @@ -7191,14 +6349,14 @@ public getUserLocalTempFolder_result( /** * Performs a deep copy on other. */ - public getUserLocalTempFolder_result(getUserLocalTempFolder_result other) { + public getUserTempFolder_result(getUserTempFolder_result other) { if (other.isSetSuccess()) { this.success = other.success; } } - public getUserLocalTempFolder_result deepCopy() { - return new getUserLocalTempFolder_result(this); + public getUserTempFolder_result deepCopy() { + return new getUserTempFolder_result(this); } @Override @@ -7210,7 +6368,7 @@ public String getSuccess() { return this.success; } - public getUserLocalTempFolder_result setSuccess(String success) { + public getUserTempFolder_result setSuccess(String success) { this.success = success; return this; } @@ -7269,12 +6427,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof getUserLocalTempFolder_result) - return this.equals((getUserLocalTempFolder_result)that); + if (that instanceof getUserTempFolder_result) + return this.equals((getUserTempFolder_result)that); return false; } - public boolean equals(getUserLocalTempFolder_result that) { + public boolean equals(getUserTempFolder_result that) { if (that == null) return false; @@ -7296,7 +6454,7 @@ public int hashCode() { } @Override - public int compareTo(getUserLocalTempFolder_result other) { + public int compareTo(getUserTempFolder_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -7330,7 +6488,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("getUserLocalTempFolder_result("); + StringBuilder sb = new StringBuilder("getUserTempFolder_result("); boolean first = true; sb.append("success:"); @@ -7365,15 +6523,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class getUserLocalTempFolder_resultStandardSchemeFactory implements SchemeFactory { - public getUserLocalTempFolder_resultStandardScheme getScheme() { - return new getUserLocalTempFolder_resultStandardScheme(); + private static class getUserTempFolder_resultStandardSchemeFactory implements SchemeFactory { + public getUserTempFolder_resultStandardScheme getScheme() { + return new getUserTempFolder_resultStandardScheme(); } } - private static class getUserLocalTempFolder_resultStandardScheme extends StandardScheme { + private static class getUserTempFolder_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, getUserLocalTempFolder_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTempFolder_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -7402,7 +6560,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, getUserLocalTempFol struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, getUserLocalTempFolder_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTempFolder_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -7417,16 +6575,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, getUserLocalTempFo } - private static class getUserLocalTempFolder_resultTupleSchemeFactory implements SchemeFactory { - public getUserLocalTempFolder_resultTupleScheme getScheme() { - return new getUserLocalTempFolder_resultTupleScheme(); + private static class getUserTempFolder_resultTupleSchemeFactory implements SchemeFactory { + public getUserTempFolder_resultTupleScheme getScheme() { + return new getUserTempFolder_resultTupleScheme(); } } - private static class getUserLocalTempFolder_resultTupleScheme extends TupleScheme { + private static class getUserTempFolder_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, getUserLocalTempFolder_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTempFolder_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetSuccess()) { @@ -7439,7 +6597,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, getUserLocalTempFol } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, getUserLocalTempFolder_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTempFolder_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { @@ -8606,8 +7764,6 @@ public void read(org.apache.thrift.protocol.TProtocol prot, lockBlock_args struc public static class lockBlock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("lockBlock_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift.protocol.TField E_P_FIELD_DESC = new org.apache.thrift.protocol.TField("eP", org.apache.thrift.protocol.TType.STRUCT, (short)1); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -8615,13 +7771,10 @@ public static class lockBlock_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -8636,10 +7789,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E_P - return E_P; default: return null; } @@ -8678,15 +7827,9 @@ public String getFieldName() { return _fieldName; } } - - // isset id assignments public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientLocationInfo.class))); - tmpMap.put(_Fields.E_P, new org.apache.thrift.meta_data.FieldMetaData("eP", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(lockBlock_result.class, metaDataMap); } @@ -8694,25 +7837,10 @@ public String getFieldName() { public lockBlock_result() { } - public lockBlock_result( - ClientLocationInfo success, - FileDoesNotExistException eP) - { - this(); - this.success = success; - this.eP = eP; - } - /** * Performs a deep copy on other. */ public lockBlock_result(lockBlock_result other) { - if (other.isSetSuccess()) { - this.success = new ClientLocationInfo(other.success); - } - if (other.isSetEP()) { - this.eP = new FileDoesNotExistException(other.eP); - } } public lockBlock_result deepCopy() { @@ -8721,87 +7849,15 @@ public lockBlock_result deepCopy() { @Override public void clear() { - this.success = null; - this.eP = null; - } - - public ClientLocationInfo getSuccess() { - return this.success; - } - - public lockBlock_result setSuccess(ClientLocationInfo success) { - this.success = success; - return this; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public FileDoesNotExistException getEP() { - return this.eP; - } - - public lockBlock_result setEP(FileDoesNotExistException eP) { - this.eP = eP; - return this; - } - - public void unsetEP() { - this.eP = null; - } - - /** Returns true if field eP is set (has been assigned a value) and false otherwise */ - public boolean isSetEP() { - return this.eP != null; - } - - public void setEPIsSet(boolean value) { - if (!value) { - this.eP = null; - } } public void setFieldValue(_Fields field, Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((ClientLocationInfo)value); - } - break; - - case E_P: - if (value == null) { - unsetEP(); - } else { - setEP((FileDoesNotExistException)value); - } - break; - } } public Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return getSuccess(); - - case E_P: - return getEP(); - } throw new IllegalStateException(); } @@ -8813,10 +7869,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); - case E_P: - return isSetEP(); } throw new IllegalStateException(); } @@ -8834,24 +7886,6 @@ public boolean equals(lockBlock_result that) { if (that == null) return false; - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_eP = true && this.isSetEP(); - boolean that_present_eP = true && that.isSetEP(); - if (this_present_eP || that_present_eP) { - if (!(this_present_eP && that_present_eP)) - return false; - if (!this.eP.equals(that.eP)) - return false; - } - return true; } @@ -8868,26 +7902,6 @@ public int compareTo(lockBlock_result other) { int lastComparison = 0; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetEP()).compareTo(other.isSetEP()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetEP()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eP, other.eP); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -8908,21 +7922,6 @@ public String toString() { StringBuilder sb = new StringBuilder("lockBlock_result("); boolean first = true; - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("eP:"); - if (this.eP == null) { - sb.append("null"); - } else { - sb.append(this.eP); - } - first = false; sb.append(")"); return sb.toString(); } @@ -8930,9 +7929,6 @@ public String toString() { public void validate() throws org.apache.thrift.TException { // check for required fields // check for sub-struct validity - if (success != null) { - success.validate(); - } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -8969,24 +7965,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, lockBlock_result st break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new ClientLocationInfo(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 1: // E_P - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.eP = new FileDoesNotExistException(); - struct.eP.read(iprot); - struct.setEPIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -9002,16 +7980,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, lockBlock_result s struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.eP != null) { - oprot.writeFieldBegin(E_P_FIELD_DESC); - struct.eP.write(oprot); - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -9029,60 +7997,35 @@ private static class lockBlock_resultTupleScheme extends TupleScheme, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("promoteBlock_args"); + public static class returnSpace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("returnSpace_args"); private static final org.apache.thrift.protocol.TField USER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("userId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField BLOCK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("blockId", org.apache.thrift.protocol.TType.I64, (short)2); + private static final org.apache.thrift.protocol.TField RETURNED_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("returnedBytes", org.apache.thrift.protocol.TType.I64, (short)2); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new promoteBlock_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new promoteBlock_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new returnSpace_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new returnSpace_argsTupleSchemeFactory()); } public long userId; // required - public long blockId; // required + public long returnedBytes; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { USER_ID((short)1, "userId"), - BLOCK_ID((short)2, "blockId"); + RETURNED_BYTES((short)2, "returnedBytes"); private static final Map byName = new HashMap(); @@ -9099,8 +8042,8 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // USER_ID return USER_ID; - case 2: // BLOCK_ID - return BLOCK_ID; + case 2: // RETURNED_BYTES + return RETURNED_BYTES; default: return null; } @@ -9142,59 +8085,59 @@ public String getFieldName() { // isset id assignments private static final int __USERID_ISSET_ID = 0; - private static final int __BLOCKID_ISSET_ID = 1; + private static final int __RETURNEDBYTES_ISSET_ID = 1; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.BLOCK_ID, new org.apache.thrift.meta_data.FieldMetaData("blockId", org.apache.thrift.TFieldRequirementType.DEFAULT, + tmpMap.put(_Fields.RETURNED_BYTES, new org.apache.thrift.meta_data.FieldMetaData("returnedBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(promoteBlock_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(returnSpace_args.class, metaDataMap); } - public promoteBlock_args() { + public returnSpace_args() { } - public promoteBlock_args( + public returnSpace_args( long userId, - long blockId) + long returnedBytes) { this(); this.userId = userId; setUserIdIsSet(true); - this.blockId = blockId; - setBlockIdIsSet(true); + this.returnedBytes = returnedBytes; + setReturnedBytesIsSet(true); } /** * Performs a deep copy on other. */ - public promoteBlock_args(promoteBlock_args other) { + public returnSpace_args(returnSpace_args other) { __isset_bitfield = other.__isset_bitfield; this.userId = other.userId; - this.blockId = other.blockId; + this.returnedBytes = other.returnedBytes; } - public promoteBlock_args deepCopy() { - return new promoteBlock_args(this); + public returnSpace_args deepCopy() { + return new returnSpace_args(this); } @Override public void clear() { setUserIdIsSet(false); this.userId = 0; - setBlockIdIsSet(false); - this.blockId = 0; + setReturnedBytesIsSet(false); + this.returnedBytes = 0; } public long getUserId() { return this.userId; } - public promoteBlock_args setUserId(long userId) { + public returnSpace_args setUserId(long userId) { this.userId = userId; setUserIdIsSet(true); return this; @@ -9213,27 +8156,27 @@ public void setUserIdIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); } - public long getBlockId() { - return this.blockId; + public long getReturnedBytes() { + return this.returnedBytes; } - public promoteBlock_args setBlockId(long blockId) { - this.blockId = blockId; - setBlockIdIsSet(true); + public returnSpace_args setReturnedBytes(long returnedBytes) { + this.returnedBytes = returnedBytes; + setReturnedBytesIsSet(true); return this; } - public void unsetBlockId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BLOCKID_ISSET_ID); + public void unsetReturnedBytes() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID); } - /** Returns true if field blockId is set (has been assigned a value) and false otherwise */ - public boolean isSetBlockId() { - return EncodingUtils.testBit(__isset_bitfield, __BLOCKID_ISSET_ID); + /** Returns true if field returnedBytes is set (has been assigned a value) and false otherwise */ + public boolean isSetReturnedBytes() { + return EncodingUtils.testBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID); } - public void setBlockIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BLOCKID_ISSET_ID, value); + public void setReturnedBytesIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -9246,11 +8189,11 @@ public void setFieldValue(_Fields field, Object value) { } break; - case BLOCK_ID: + case RETURNED_BYTES: if (value == null) { - unsetBlockId(); + unsetReturnedBytes(); } else { - setBlockId((Long)value); + setReturnedBytes((Long)value); } break; @@ -9262,8 +8205,8 @@ public Object getFieldValue(_Fields field) { case USER_ID: return Long.valueOf(getUserId()); - case BLOCK_ID: - return Long.valueOf(getBlockId()); + case RETURNED_BYTES: + return Long.valueOf(getReturnedBytes()); } throw new IllegalStateException(); @@ -9278,8 +8221,8 @@ public boolean isSet(_Fields field) { switch (field) { case USER_ID: return isSetUserId(); - case BLOCK_ID: - return isSetBlockId(); + case RETURNED_BYTES: + return isSetReturnedBytes(); } throw new IllegalStateException(); } @@ -9288,12 +8231,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof promoteBlock_args) - return this.equals((promoteBlock_args)that); + if (that instanceof returnSpace_args) + return this.equals((returnSpace_args)that); return false; } - public boolean equals(promoteBlock_args that) { + public boolean equals(returnSpace_args that) { if (that == null) return false; @@ -9306,12 +8249,12 @@ public boolean equals(promoteBlock_args that) { return false; } - boolean this_present_blockId = true; - boolean that_present_blockId = true; - if (this_present_blockId || that_present_blockId) { - if (!(this_present_blockId && that_present_blockId)) + boolean this_present_returnedBytes = true; + boolean that_present_returnedBytes = true; + if (this_present_returnedBytes || that_present_returnedBytes) { + if (!(this_present_returnedBytes && that_present_returnedBytes)) return false; - if (this.blockId != that.blockId) + if (this.returnedBytes != that.returnedBytes) return false; } @@ -9324,7 +8267,7 @@ public int hashCode() { } @Override - public int compareTo(promoteBlock_args other) { + public int compareTo(returnSpace_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -9341,12 +8284,12 @@ public int compareTo(promoteBlock_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetBlockId()).compareTo(other.isSetBlockId()); + lastComparison = Boolean.valueOf(isSetReturnedBytes()).compareTo(other.isSetReturnedBytes()); if (lastComparison != 0) { return lastComparison; } - if (isSetBlockId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.blockId, other.blockId); + if (isSetReturnedBytes()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.returnedBytes, other.returnedBytes); if (lastComparison != 0) { return lastComparison; } @@ -9368,15 +8311,15 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("promoteBlock_args("); + StringBuilder sb = new StringBuilder("returnSpace_args("); boolean first = true; sb.append("userId:"); sb.append(this.userId); first = false; if (!first) sb.append(", "); - sb.append("blockId:"); - sb.append(this.blockId); + sb.append("returnedBytes:"); + sb.append(this.returnedBytes); first = false; sb.append(")"); return sb.toString(); @@ -9405,15 +8348,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class promoteBlock_argsStandardSchemeFactory implements SchemeFactory { - public promoteBlock_argsStandardScheme getScheme() { - return new promoteBlock_argsStandardScheme(); + private static class returnSpace_argsStandardSchemeFactory implements SchemeFactory { + public returnSpace_argsStandardScheme getScheme() { + return new returnSpace_argsStandardScheme(); } } - private static class promoteBlock_argsStandardScheme extends StandardScheme { + private static class returnSpace_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, returnSpace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -9431,10 +8374,10 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_args s org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // BLOCK_ID + case 2: // RETURNED_BYTES if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.blockId = iprot.readI64(); - struct.setBlockIdIsSet(true); + struct.returnedBytes = iprot.readI64(); + struct.setReturnedBytesIsSet(true); } else { org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -9450,15 +8393,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_args s struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, promoteBlock_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, returnSpace_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldBegin(USER_ID_FIELD_DESC); oprot.writeI64(struct.userId); oprot.writeFieldEnd(); - oprot.writeFieldBegin(BLOCK_ID_FIELD_DESC); - oprot.writeI64(struct.blockId); + oprot.writeFieldBegin(RETURNED_BYTES_FIELD_DESC); + oprot.writeI64(struct.returnedBytes); oprot.writeFieldEnd(); oprot.writeFieldStop(); oprot.writeStructEnd(); @@ -9466,35 +8409,35 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, promoteBlock_args } - private static class promoteBlock_argsTupleSchemeFactory implements SchemeFactory { - public promoteBlock_argsTupleScheme getScheme() { - return new promoteBlock_argsTupleScheme(); + private static class returnSpace_argsTupleSchemeFactory implements SchemeFactory { + public returnSpace_argsTupleScheme getScheme() { + return new returnSpace_argsTupleScheme(); } } - private static class promoteBlock_argsTupleScheme extends TupleScheme { + private static class returnSpace_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, promoteBlock_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, returnSpace_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetUserId()) { optionals.set(0); } - if (struct.isSetBlockId()) { + if (struct.isSetReturnedBytes()) { optionals.set(1); } oprot.writeBitSet(optionals, 2); if (struct.isSetUserId()) { oprot.writeI64(struct.userId); } - if (struct.isSetBlockId()) { - oprot.writeI64(struct.blockId); + if (struct.isSetReturnedBytes()) { + oprot.writeI64(struct.returnedBytes); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, promoteBlock_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, returnSpace_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { @@ -9502,30 +8445,28 @@ public void read(org.apache.thrift.protocol.TProtocol prot, promoteBlock_args st struct.setUserIdIsSet(true); } if (incoming.get(1)) { - struct.blockId = iprot.readI64(); - struct.setBlockIdIsSet(true); + struct.returnedBytes = iprot.readI64(); + struct.setReturnedBytesIsSet(true); } } } } - public static class promoteBlock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("promoteBlock_result"); + public static class returnSpace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("returnSpace_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new promoteBlock_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new promoteBlock_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new returnSpace_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new returnSpace_resultTupleSchemeFactory()); } - public boolean success; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"); +; private static final Map byName = new HashMap(); @@ -9540,8 +8481,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; default: return null; } @@ -9580,89 +8519,37 @@ public String getFieldName() { return _fieldName; } } - - // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(promoteBlock_result.class, metaDataMap); - } - - public promoteBlock_result() { + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(returnSpace_result.class, metaDataMap); } - public promoteBlock_result( - boolean success) - { - this(); - this.success = success; - setSuccessIsSet(true); + public returnSpace_result() { } /** * Performs a deep copy on other. */ - public promoteBlock_result(promoteBlock_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; + public returnSpace_result(returnSpace_result other) { } - public promoteBlock_result deepCopy() { - return new promoteBlock_result(this); + public returnSpace_result deepCopy() { + return new returnSpace_result(this); } @Override public void clear() { - setSuccessIsSet(false); - this.success = false; - } - - public boolean isSuccess() { - return this.success; - } - - public promoteBlock_result setSuccess(boolean success) { - this.success = success; - setSuccessIsSet(true); - return this; - } - - public void unsetSuccess() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - public void setSuccessIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((Boolean)value); - } - break; - } } public Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return Boolean.valueOf(isSuccess()); - } throw new IllegalStateException(); } @@ -9674,8 +8561,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); } throw new IllegalStateException(); } @@ -9684,24 +8569,15 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof promoteBlock_result) - return this.equals((promoteBlock_result)that); + if (that instanceof returnSpace_result) + return this.equals((returnSpace_result)that); return false; } - public boolean equals(promoteBlock_result that) { + public boolean equals(returnSpace_result that) { if (that == null) return false; - boolean this_present_success = true; - boolean that_present_success = true; - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (this.success != that.success) - return false; - } - return true; } @@ -9711,23 +8587,13 @@ public int hashCode() { } @Override - public int compareTo(promoteBlock_result other) { + public int compareTo(returnSpace_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -9745,12 +8611,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("promoteBlock_result("); + StringBuilder sb = new StringBuilder("returnSpace_result("); boolean first = true; - sb.append("success:"); - sb.append(this.success); - first = false; sb.append(")"); return sb.toString(); } @@ -9770,23 +8633,21 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); } } - private static class promoteBlock_resultStandardSchemeFactory implements SchemeFactory { - public promoteBlock_resultStandardScheme getScheme() { - return new promoteBlock_resultStandardScheme(); + private static class returnSpace_resultStandardSchemeFactory implements SchemeFactory { + public returnSpace_resultStandardScheme getScheme() { + return new returnSpace_resultStandardScheme(); } } - private static class promoteBlock_resultStandardScheme extends StandardScheme { + private static class returnSpace_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, returnSpace_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -9796,14 +8657,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_result break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.success = iprot.readBool(); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -9813,1772 +8666,58 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, promoteBlock_result // check for required fields of primitive type, which can't be checked in the validate method struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, promoteBlock_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class promoteBlock_resultTupleSchemeFactory implements SchemeFactory { - public promoteBlock_resultTupleScheme getScheme() { - return new promoteBlock_resultTupleScheme(); - } - } - - private static class promoteBlock_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, promoteBlock_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - oprot.writeBitSet(optionals, 1); - if (struct.isSetSuccess()) { - oprot.writeBool(struct.success); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, promoteBlock_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(1); - if (incoming.get(0)) { - struct.success = iprot.readBool(); - struct.setSuccessIsSet(true); - } - } - } - - } - - public static class returnSpace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("returnSpace_args"); - - private static final org.apache.thrift.protocol.TField USER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("userId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField STORAGE_DIR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storageDirId", org.apache.thrift.protocol.TType.I64, (short)2); - private static final org.apache.thrift.protocol.TField RETURNED_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("returnedBytes", org.apache.thrift.protocol.TType.I64, (short)3); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new returnSpace_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new returnSpace_argsTupleSchemeFactory()); - } - - public long userId; // required - public long storageDirId; // required - public long returnedBytes; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - USER_ID((short)1, "userId"), - STORAGE_DIR_ID((short)2, "storageDirId"), - RETURNED_BYTES((short)3, "returnedBytes"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // USER_ID - return USER_ID; - case 2: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 3: // RETURNED_BYTES - return RETURNED_BYTES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __USERID_ISSET_ID = 0; - private static final int __STORAGEDIRID_ISSET_ID = 1; - private static final int __RETURNEDBYTES_ISSET_ID = 2; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.RETURNED_BYTES, new org.apache.thrift.meta_data.FieldMetaData("returnedBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(returnSpace_args.class, metaDataMap); - } - - public returnSpace_args() { - } - - public returnSpace_args( - long userId, - long storageDirId, - long returnedBytes) - { - this(); - this.userId = userId; - setUserIdIsSet(true); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - this.returnedBytes = returnedBytes; - setReturnedBytesIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public returnSpace_args(returnSpace_args other) { - __isset_bitfield = other.__isset_bitfield; - this.userId = other.userId; - this.storageDirId = other.storageDirId; - this.returnedBytes = other.returnedBytes; - } - - public returnSpace_args deepCopy() { - return new returnSpace_args(this); - } - - @Override - public void clear() { - setUserIdIsSet(false); - this.userId = 0; - setStorageDirIdIsSet(false); - this.storageDirId = 0; - setReturnedBytesIsSet(false); - this.returnedBytes = 0; - } - - public long getUserId() { - return this.userId; - } - - public returnSpace_args setUserId(long userId) { - this.userId = userId; - setUserIdIsSet(true); - return this; - } - - public void unsetUserId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USERID_ISSET_ID); - } - - /** Returns true if field userId is set (has been assigned a value) and false otherwise */ - public boolean isSetUserId() { - return EncodingUtils.testBit(__isset_bitfield, __USERID_ISSET_ID); - } - - public void setUserIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); - } - - public long getStorageDirId() { - return this.storageDirId; - } - - public returnSpace_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - - public long getReturnedBytes() { - return this.returnedBytes; - } - - public returnSpace_args setReturnedBytes(long returnedBytes) { - this.returnedBytes = returnedBytes; - setReturnedBytesIsSet(true); - return this; - } - - public void unsetReturnedBytes() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID); - } - - /** Returns true if field returnedBytes is set (has been assigned a value) and false otherwise */ - public boolean isSetReturnedBytes() { - return EncodingUtils.testBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID); - } - - public void setReturnedBytesIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RETURNEDBYTES_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case USER_ID: - if (value == null) { - unsetUserId(); - } else { - setUserId((Long)value); - } - break; - - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - - case RETURNED_BYTES: - if (value == null) { - unsetReturnedBytes(); - } else { - setReturnedBytes((Long)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case USER_ID: - return Long.valueOf(getUserId()); - - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - - case RETURNED_BYTES: - return Long.valueOf(getReturnedBytes()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case USER_ID: - return isSetUserId(); - case STORAGE_DIR_ID: - return isSetStorageDirId(); - case RETURNED_BYTES: - return isSetReturnedBytes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof returnSpace_args) - return this.equals((returnSpace_args)that); - return false; - } - - public boolean equals(returnSpace_args that) { - if (that == null) - return false; - - boolean this_present_userId = true; - boolean that_present_userId = true; - if (this_present_userId || that_present_userId) { - if (!(this_present_userId && that_present_userId)) - return false; - if (this.userId != that.userId) - return false; - } - - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - - boolean this_present_returnedBytes = true; - boolean that_present_returnedBytes = true; - if (this_present_returnedBytes || that_present_returnedBytes) { - if (!(this_present_returnedBytes && that_present_returnedBytes)) - return false; - if (this.returnedBytes != that.returnedBytes) - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public int compareTo(returnSpace_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - - lastComparison = Boolean.valueOf(isSetUserId()).compareTo(other.isSetUserId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetUserId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userId, other.userId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetReturnedBytes()).compareTo(other.isSetReturnedBytes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetReturnedBytes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.returnedBytes, other.returnedBytes); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("returnSpace_args("); - boolean first = true; - - sb.append("userId:"); - sb.append(this.userId); - first = false; - if (!first) sb.append(", "); - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); - sb.append("returnedBytes:"); - sb.append(this.returnedBytes); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class returnSpace_argsStandardSchemeFactory implements SchemeFactory { - public returnSpace_argsStandardScheme getScheme() { - return new returnSpace_argsStandardScheme(); - } - } - - private static class returnSpace_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, returnSpace_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // USER_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.userId = iprot.readI64(); - struct.setUserIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // RETURNED_BYTES - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.returnedBytes = iprot.readI64(); - struct.setReturnedBytesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - // check for required fields of primitive type, which can't be checked in the validate method - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, returnSpace_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(USER_ID_FIELD_DESC); - oprot.writeI64(struct.userId); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(RETURNED_BYTES_FIELD_DESC); - oprot.writeI64(struct.returnedBytes); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class returnSpace_argsTupleSchemeFactory implements SchemeFactory { - public returnSpace_argsTupleScheme getScheme() { - return new returnSpace_argsTupleScheme(); - } - } - - private static class returnSpace_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, returnSpace_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetUserId()) { - optionals.set(0); - } - if (struct.isSetStorageDirId()) { - optionals.set(1); - } - if (struct.isSetReturnedBytes()) { - optionals.set(2); - } - oprot.writeBitSet(optionals, 3); - if (struct.isSetUserId()) { - oprot.writeI64(struct.userId); - } - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } - if (struct.isSetReturnedBytes()) { - oprot.writeI64(struct.returnedBytes); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, returnSpace_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); - if (incoming.get(0)) { - struct.userId = iprot.readI64(); - struct.setUserIdIsSet(true); - } - if (incoming.get(1)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } - if (incoming.get(2)) { - struct.returnedBytes = iprot.readI64(); - struct.setReturnedBytesIsSet(true); - } - } - } - - } - - public static class returnSpace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("returnSpace_result"); - - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new returnSpace_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new returnSpace_resultTupleSchemeFactory()); - } - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(returnSpace_result.class, metaDataMap); - } - - public returnSpace_result() { - } - - /** - * Performs a deep copy on other. - */ - public returnSpace_result(returnSpace_result other) { - } - - public returnSpace_result deepCopy() { - return new returnSpace_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof returnSpace_result) - return this.equals((returnSpace_result)that); - return false; - } - - public boolean equals(returnSpace_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public int compareTo(returnSpace_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("returnSpace_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class returnSpace_resultStandardSchemeFactory implements SchemeFactory { - public returnSpace_resultStandardScheme getScheme() { - return new returnSpace_resultStandardScheme(); - } - } - - private static class returnSpace_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, returnSpace_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - // check for required fields of primitive type, which can't be checked in the validate method - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, returnSpace_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class returnSpace_resultTupleSchemeFactory implements SchemeFactory { - public returnSpace_resultTupleScheme getScheme() { - return new returnSpace_resultTupleScheme(); - } - } - - private static class returnSpace_resultTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, returnSpace_result struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, returnSpace_result struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - } - } - - } - - public static class requestSpace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpace_args"); - - private static final org.apache.thrift.protocol.TField USER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("userId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField REQUEST_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("requestBytes", org.apache.thrift.protocol.TType.I64, (short)2); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new requestSpace_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new requestSpace_argsTupleSchemeFactory()); - } - - public long userId; // required - public long requestBytes; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - USER_ID((short)1, "userId"), - REQUEST_BYTES((short)2, "requestBytes"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // USER_ID - return USER_ID; - case 2: // REQUEST_BYTES - return REQUEST_BYTES; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __USERID_ISSET_ID = 0; - private static final int __REQUESTBYTES_ISSET_ID = 1; - private byte __isset_bitfield = 0; - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.REQUEST_BYTES, new org.apache.thrift.meta_data.FieldMetaData("requestBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpace_args.class, metaDataMap); - } - - public requestSpace_args() { - } - - public requestSpace_args( - long userId, - long requestBytes) - { - this(); - this.userId = userId; - setUserIdIsSet(true); - this.requestBytes = requestBytes; - setRequestBytesIsSet(true); - } - - /** - * Performs a deep copy on other. - */ - public requestSpace_args(requestSpace_args other) { - __isset_bitfield = other.__isset_bitfield; - this.userId = other.userId; - this.requestBytes = other.requestBytes; - } - - public requestSpace_args deepCopy() { - return new requestSpace_args(this); - } - - @Override - public void clear() { - setUserIdIsSet(false); - this.userId = 0; - setRequestBytesIsSet(false); - this.requestBytes = 0; - } - - public long getUserId() { - return this.userId; - } - - public requestSpace_args setUserId(long userId) { - this.userId = userId; - setUserIdIsSet(true); - return this; - } - - public void unsetUserId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USERID_ISSET_ID); - } - - /** Returns true if field userId is set (has been assigned a value) and false otherwise */ - public boolean isSetUserId() { - return EncodingUtils.testBit(__isset_bitfield, __USERID_ISSET_ID); - } - - public void setUserIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); - } - - public long getRequestBytes() { - return this.requestBytes; - } - - public requestSpace_args setRequestBytes(long requestBytes) { - this.requestBytes = requestBytes; - setRequestBytesIsSet(true); - return this; - } - - public void unsetRequestBytes() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTBYTES_ISSET_ID); - } - - /** Returns true if field requestBytes is set (has been assigned a value) and false otherwise */ - public boolean isSetRequestBytes() { - return EncodingUtils.testBit(__isset_bitfield, __REQUESTBYTES_ISSET_ID); - } - - public void setRequestBytesIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTBYTES_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case USER_ID: - if (value == null) { - unsetUserId(); - } else { - setUserId((Long)value); - } - break; - - case REQUEST_BYTES: - if (value == null) { - unsetRequestBytes(); - } else { - setRequestBytes((Long)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case USER_ID: - return Long.valueOf(getUserId()); - - case REQUEST_BYTES: - return Long.valueOf(getRequestBytes()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case USER_ID: - return isSetUserId(); - case REQUEST_BYTES: - return isSetRequestBytes(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof requestSpace_args) - return this.equals((requestSpace_args)that); - return false; - } - - public boolean equals(requestSpace_args that) { - if (that == null) - return false; - - boolean this_present_userId = true; - boolean that_present_userId = true; - if (this_present_userId || that_present_userId) { - if (!(this_present_userId && that_present_userId)) - return false; - if (this.userId != that.userId) - return false; - } - - boolean this_present_requestBytes = true; - boolean that_present_requestBytes = true; - if (this_present_requestBytes || that_present_requestBytes) { - if (!(this_present_requestBytes && that_present_requestBytes)) - return false; - if (this.requestBytes != that.requestBytes) - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public int compareTo(requestSpace_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - - lastComparison = Boolean.valueOf(isSetUserId()).compareTo(other.isSetUserId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetUserId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userId, other.userId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetRequestBytes()).compareTo(other.isSetRequestBytes()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetRequestBytes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requestBytes, other.requestBytes); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("requestSpace_args("); - boolean first = true; - - sb.append("userId:"); - sb.append(this.userId); - first = false; - if (!first) sb.append(", "); - sb.append("requestBytes:"); - sb.append(this.requestBytes); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class requestSpace_argsStandardSchemeFactory implements SchemeFactory { - public requestSpace_argsStandardScheme getScheme() { - return new requestSpace_argsStandardScheme(); - } - } - - private static class requestSpace_argsStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpace_args struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 1: // USER_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.userId = iprot.readI64(); - struct.setUserIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 2: // REQUEST_BYTES - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.requestBytes = iprot.readI64(); - struct.setRequestBytesIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - // check for required fields of primitive type, which can't be checked in the validate method - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpace_args struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(USER_ID_FIELD_DESC); - oprot.writeI64(struct.userId); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(REQUEST_BYTES_FIELD_DESC); - oprot.writeI64(struct.requestBytes); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - } - - private static class requestSpace_argsTupleSchemeFactory implements SchemeFactory { - public requestSpace_argsTupleScheme getScheme() { - return new requestSpace_argsTupleScheme(); - } - } - - private static class requestSpace_argsTupleScheme extends TupleScheme { - - @Override - public void write(org.apache.thrift.protocol.TProtocol prot, requestSpace_args struct) throws org.apache.thrift.TException { - TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetUserId()) { - optionals.set(0); - } - if (struct.isSetRequestBytes()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetUserId()) { - oprot.writeI64(struct.userId); - } - if (struct.isSetRequestBytes()) { - oprot.writeI64(struct.requestBytes); - } - } - - @Override - public void read(org.apache.thrift.protocol.TProtocol prot, requestSpace_args struct) throws org.apache.thrift.TException { - TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.userId = iprot.readI64(); - struct.setUserIdIsSet(true); - } - if (incoming.get(1)) { - struct.requestBytes = iprot.readI64(); - struct.setRequestBytesIsSet(true); - } - } - } - - } - - public static class requestSpace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpace_result"); - - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift.protocol.TField E_P_FIELD_DESC = new org.apache.thrift.protocol.TField("eP", org.apache.thrift.protocol.TType.STRUCT, (short)1); - - private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); - static { - schemes.put(StandardScheme.class, new requestSpace_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new requestSpace_resultTupleSchemeFactory()); - } - - public ClientLocationInfo success; // required - public OutOfSpaceException eP; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift.TFieldIdEnum { - SUCCESS((short)0, "success"), - E_P((short)1, "eP"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E_P - return E_P; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientLocationInfo.class))); - tmpMap.put(_Fields.E_P, new org.apache.thrift.meta_data.FieldMetaData("eP", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpace_result.class, metaDataMap); - } - - public requestSpace_result() { - } - - public requestSpace_result( - ClientLocationInfo success, - OutOfSpaceException eP) - { - this(); - this.success = success; - this.eP = eP; - } - - /** - * Performs a deep copy on other. - */ - public requestSpace_result(requestSpace_result other) { - if (other.isSetSuccess()) { - this.success = new ClientLocationInfo(other.success); - } - if (other.isSetEP()) { - this.eP = new OutOfSpaceException(other.eP); - } - } - - public requestSpace_result deepCopy() { - return new requestSpace_result(this); - } - - @Override - public void clear() { - this.success = null; - this.eP = null; - } - - public ClientLocationInfo getSuccess() { - return this.success; - } - - public requestSpace_result setSuccess(ClientLocationInfo success) { - this.success = success; - return this; - } - - public void unsetSuccess() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return this.success != null; - } - - public void setSuccessIsSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public OutOfSpaceException getEP() { - return this.eP; - } - - public requestSpace_result setEP(OutOfSpaceException eP) { - this.eP = eP; - return this; - } - - public void unsetEP() { - this.eP = null; - } - - /** Returns true if field eP is set (has been assigned a value) and false otherwise */ - public boolean isSetEP() { - return this.eP != null; - } - - public void setEPIsSet(boolean value) { - if (!value) { - this.eP = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((ClientLocationInfo)value); - } - break; - - case E_P: - if (value == null) { - unsetEP(); - } else { - setEP((OutOfSpaceException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return getSuccess(); - - case E_P: - return getEP(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return isSetSuccess(); - case E_P: - return isSetEP(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof requestSpace_result) - return this.equals((requestSpace_result)that); - return false; - } - - public boolean equals(requestSpace_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.isSetSuccess(); - boolean that_present_success = true && that.isSetSuccess(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_eP = true && this.isSetEP(); - boolean that_present_eP = true && that.isSetEP(); - if (this_present_eP || that_present_eP) { - if (!(this_present_eP && that_present_eP)) - return false; - if (!this.eP.equals(that.eP)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public int compareTo(requestSpace_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(isSetEP()).compareTo(other.isSetEP()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetEP()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eP, other.eP); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { - schemes.get(iprot.getScheme()).getScheme().read(iprot, this); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { - schemes.get(oprot.getScheme()).getScheme().write(oprot, this); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("requestSpace_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("eP:"); - if (this.eP == null) { - sb.append("null"); - } else { - sb.append(this.eP); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift.TException { - // check for required fields - // check for sub-struct validity - if (success != null) { - success.validate(); - } - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift.TException te) { - throw new java.io.IOException(te); - } - } - - private static class requestSpace_resultStandardSchemeFactory implements SchemeFactory { - public requestSpace_resultStandardScheme getScheme() { - return new requestSpace_resultStandardScheme(); - } - } - - private static class requestSpace_resultStandardScheme extends StandardScheme { - - public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpace_result struct) throws org.apache.thrift.TException { - org.apache.thrift.protocol.TField schemeField; - iprot.readStructBegin(); - while (true) - { - schemeField = iprot.readFieldBegin(); - if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { - break; - } - switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.success = new ClientLocationInfo(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 1: // E_P - if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { - struct.eP = new OutOfSpaceException(); - struct.eP.read(iprot); - struct.setEPIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - default: - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - - // check for required fields of primitive type, which can't be checked in the validate method - struct.validate(); - } - - public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpace_result struct) throws org.apache.thrift.TException { - struct.validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (struct.success != null) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - struct.success.write(oprot); - oprot.writeFieldEnd(); - } - if (struct.eP != null) { - oprot.writeFieldBegin(E_P_FIELD_DESC); - struct.eP.write(oprot); - oprot.writeFieldEnd(); - } + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, returnSpace_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldStop(); oprot.writeStructEnd(); } } - private static class requestSpace_resultTupleSchemeFactory implements SchemeFactory { - public requestSpace_resultTupleScheme getScheme() { - return new requestSpace_resultTupleScheme(); + private static class returnSpace_resultTupleSchemeFactory implements SchemeFactory { + public returnSpace_resultTupleScheme getScheme() { + return new returnSpace_resultTupleScheme(); } } - private static class requestSpace_resultTupleScheme extends TupleScheme { + private static class returnSpace_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, requestSpace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, returnSpace_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; - BitSet optionals = new BitSet(); - if (struct.isSetSuccess()) { - optionals.set(0); - } - if (struct.isSetEP()) { - optionals.set(1); - } - oprot.writeBitSet(optionals, 2); - if (struct.isSetSuccess()) { - struct.success.write(oprot); - } - if (struct.isSetEP()) { - struct.eP.write(oprot); - } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, requestSpace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, returnSpace_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(2); - if (incoming.get(0)) { - struct.success = new ClientLocationInfo(); - struct.success.read(iprot); - struct.setSuccessIsSet(true); - } - if (incoming.get(1)) { - struct.eP = new OutOfSpaceException(); - struct.eP.read(iprot); - struct.setEPIsSet(true); - } } } } - public static class requestSpaceInPlace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpaceInPlace_args"); + public static class requestSpace_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpace_args"); private static final org.apache.thrift.protocol.TField USER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("userId", org.apache.thrift.protocol.TType.I64, (short)1); - private static final org.apache.thrift.protocol.TField STORAGE_DIR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("storageDirId", org.apache.thrift.protocol.TType.I64, (short)2); - private static final org.apache.thrift.protocol.TField REQUEST_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("requestBytes", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField REQUEST_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("requestBytes", org.apache.thrift.protocol.TType.I64, (short)2); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new requestSpaceInPlace_argsStandardSchemeFactory()); - schemes.put(TupleScheme.class, new requestSpaceInPlace_argsTupleSchemeFactory()); + schemes.put(StandardScheme.class, new requestSpace_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new requestSpace_argsTupleSchemeFactory()); } public long userId; // required - public long storageDirId; // required public long requestBytes; // required /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { USER_ID((short)1, "userId"), - STORAGE_DIR_ID((short)2, "storageDirId"), - REQUEST_BYTES((short)3, "requestBytes"); + REQUEST_BYTES((short)2, "requestBytes"); private static final Map byName = new HashMap(); @@ -11595,9 +8734,7 @@ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { case 1: // USER_ID return USER_ID; - case 2: // STORAGE_DIR_ID - return STORAGE_DIR_ID; - case 3: // REQUEST_BYTES + case 2: // REQUEST_BYTES return REQUEST_BYTES; default: return null; @@ -11640,35 +8777,29 @@ public String getFieldName() { // isset id assignments private static final int __USERID_ISSET_ID = 0; - private static final int __STORAGEDIRID_ISSET_ID = 1; - private static final int __REQUESTBYTES_ISSET_ID = 2; + private static final int __REQUESTBYTES_ISSET_ID = 1; private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); tmpMap.put(_Fields.USER_ID, new org.apache.thrift.meta_data.FieldMetaData("userId", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); - tmpMap.put(_Fields.STORAGE_DIR_ID, new org.apache.thrift.meta_data.FieldMetaData("storageDirId", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); tmpMap.put(_Fields.REQUEST_BYTES, new org.apache.thrift.meta_data.FieldMetaData("requestBytes", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpaceInPlace_args.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpace_args.class, metaDataMap); } - public requestSpaceInPlace_args() { + public requestSpace_args() { } - public requestSpaceInPlace_args( + public requestSpace_args( long userId, - long storageDirId, long requestBytes) { this(); this.userId = userId; setUserIdIsSet(true); - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); this.requestBytes = requestBytes; setRequestBytesIsSet(true); } @@ -11676,23 +8807,20 @@ public requestSpaceInPlace_args( /** * Performs a deep copy on other. */ - public requestSpaceInPlace_args(requestSpaceInPlace_args other) { + public requestSpace_args(requestSpace_args other) { __isset_bitfield = other.__isset_bitfield; this.userId = other.userId; - this.storageDirId = other.storageDirId; this.requestBytes = other.requestBytes; } - public requestSpaceInPlace_args deepCopy() { - return new requestSpaceInPlace_args(this); + public requestSpace_args deepCopy() { + return new requestSpace_args(this); } @Override public void clear() { setUserIdIsSet(false); this.userId = 0; - setStorageDirIdIsSet(false); - this.storageDirId = 0; setRequestBytesIsSet(false); this.requestBytes = 0; } @@ -11701,7 +8829,7 @@ public long getUserId() { return this.userId; } - public requestSpaceInPlace_args setUserId(long userId) { + public requestSpace_args setUserId(long userId) { this.userId = userId; setUserIdIsSet(true); return this; @@ -11720,34 +8848,11 @@ public void setUserIdIsSet(boolean value) { __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USERID_ISSET_ID, value); } - public long getStorageDirId() { - return this.storageDirId; - } - - public requestSpaceInPlace_args setStorageDirId(long storageDirId) { - this.storageDirId = storageDirId; - setStorageDirIdIsSet(true); - return this; - } - - public void unsetStorageDirId() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - /** Returns true if field storageDirId is set (has been assigned a value) and false otherwise */ - public boolean isSetStorageDirId() { - return EncodingUtils.testBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID); - } - - public void setStorageDirIdIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STORAGEDIRID_ISSET_ID, value); - } - public long getRequestBytes() { return this.requestBytes; } - public requestSpaceInPlace_args setRequestBytes(long requestBytes) { + public requestSpace_args setRequestBytes(long requestBytes) { this.requestBytes = requestBytes; setRequestBytesIsSet(true); return this; @@ -11776,14 +8881,6 @@ public void setFieldValue(_Fields field, Object value) { } break; - case STORAGE_DIR_ID: - if (value == null) { - unsetStorageDirId(); - } else { - setStorageDirId((Long)value); - } - break; - case REQUEST_BYTES: if (value == null) { unsetRequestBytes(); @@ -11800,9 +8897,6 @@ public Object getFieldValue(_Fields field) { case USER_ID: return Long.valueOf(getUserId()); - case STORAGE_DIR_ID: - return Long.valueOf(getStorageDirId()); - case REQUEST_BYTES: return Long.valueOf(getRequestBytes()); @@ -11819,8 +8913,6 @@ public boolean isSet(_Fields field) { switch (field) { case USER_ID: return isSetUserId(); - case STORAGE_DIR_ID: - return isSetStorageDirId(); case REQUEST_BYTES: return isSetRequestBytes(); } @@ -11831,12 +8923,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof requestSpaceInPlace_args) - return this.equals((requestSpaceInPlace_args)that); + if (that instanceof requestSpace_args) + return this.equals((requestSpace_args)that); return false; } - public boolean equals(requestSpaceInPlace_args that) { + public boolean equals(requestSpace_args that) { if (that == null) return false; @@ -11849,15 +8941,6 @@ public boolean equals(requestSpaceInPlace_args that) { return false; } - boolean this_present_storageDirId = true; - boolean that_present_storageDirId = true; - if (this_present_storageDirId || that_present_storageDirId) { - if (!(this_present_storageDirId && that_present_storageDirId)) - return false; - if (this.storageDirId != that.storageDirId) - return false; - } - boolean this_present_requestBytes = true; boolean that_present_requestBytes = true; if (this_present_requestBytes || that_present_requestBytes) { @@ -11876,7 +8959,7 @@ public int hashCode() { } @Override - public int compareTo(requestSpaceInPlace_args other) { + public int compareTo(requestSpace_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -11893,16 +8976,6 @@ public int compareTo(requestSpaceInPlace_args other) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStorageDirId()).compareTo(other.isSetStorageDirId()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetStorageDirId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.storageDirId, other.storageDirId); - if (lastComparison != 0) { - return lastComparison; - } - } lastComparison = Boolean.valueOf(isSetRequestBytes()).compareTo(other.isSetRequestBytes()); if (lastComparison != 0) { return lastComparison; @@ -11930,17 +9003,13 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("requestSpaceInPlace_args("); + StringBuilder sb = new StringBuilder("requestSpace_args("); boolean first = true; sb.append("userId:"); sb.append(this.userId); first = false; if (!first) sb.append(", "); - sb.append("storageDirId:"); - sb.append(this.storageDirId); - first = false; - if (!first) sb.append(", "); sb.append("requestBytes:"); sb.append(this.requestBytes); first = false; @@ -11971,15 +9040,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class requestSpaceInPlace_argsStandardSchemeFactory implements SchemeFactory { - public requestSpaceInPlace_argsStandardScheme getScheme() { - return new requestSpaceInPlace_argsStandardScheme(); + private static class requestSpace_argsStandardSchemeFactory implements SchemeFactory { + public requestSpace_argsStandardScheme getScheme() { + return new requestSpace_argsStandardScheme(); } } - private static class requestSpaceInPlace_argsStandardScheme extends StandardScheme { + private static class requestSpace_argsStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpaceInPlace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpace_args struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -11997,15 +9066,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpaceInPlace org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; - case 2: // STORAGE_DIR_ID - if (schemeField.type == org.apache.thrift.protocol.TType.I64) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; - case 3: // REQUEST_BYTES + case 2: // REQUEST_BYTES if (schemeField.type == org.apache.thrift.protocol.TType.I64) { struct.requestBytes = iprot.readI64(); struct.setRequestBytesIsSet(true); @@ -12024,16 +9085,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpaceInPlace struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpaceInPlace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpace_args struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); oprot.writeFieldBegin(USER_ID_FIELD_DESC); oprot.writeI64(struct.userId); oprot.writeFieldEnd(); - oprot.writeFieldBegin(STORAGE_DIR_ID_FIELD_DESC); - oprot.writeI64(struct.storageDirId); - oprot.writeFieldEnd(); oprot.writeFieldBegin(REQUEST_BYTES_FIELD_DESC); oprot.writeI64(struct.requestBytes); oprot.writeFieldEnd(); @@ -12043,52 +9101,42 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpaceInPlac } - private static class requestSpaceInPlace_argsTupleSchemeFactory implements SchemeFactory { - public requestSpaceInPlace_argsTupleScheme getScheme() { - return new requestSpaceInPlace_argsTupleScheme(); + private static class requestSpace_argsTupleSchemeFactory implements SchemeFactory { + public requestSpace_argsTupleScheme getScheme() { + return new requestSpace_argsTupleScheme(); } } - private static class requestSpaceInPlace_argsTupleScheme extends TupleScheme { + private static class requestSpace_argsTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace_args struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, requestSpace_args struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetUserId()) { optionals.set(0); } - if (struct.isSetStorageDirId()) { - optionals.set(1); - } if (struct.isSetRequestBytes()) { - optionals.set(2); + optionals.set(1); } - oprot.writeBitSet(optionals, 3); + oprot.writeBitSet(optionals, 2); if (struct.isSetUserId()) { oprot.writeI64(struct.userId); } - if (struct.isSetStorageDirId()) { - oprot.writeI64(struct.storageDirId); - } if (struct.isSetRequestBytes()) { oprot.writeI64(struct.requestBytes); } } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace_args struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, requestSpace_args struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; - BitSet incoming = iprot.readBitSet(3); + BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { struct.userId = iprot.readI64(); struct.setUserIdIsSet(true); } if (incoming.get(1)) { - struct.storageDirId = iprot.readI64(); - struct.setStorageDirIdIsSet(true); - } - if (incoming.get(2)) { struct.requestBytes = iprot.readI64(); struct.setRequestBytesIsSet(true); } @@ -12097,15 +9145,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace_ } - public static class requestSpaceInPlace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { - private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpaceInPlace_result"); + public static class requestSpace_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("requestSpace_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { - schemes.put(StandardScheme.class, new requestSpaceInPlace_resultStandardSchemeFactory()); - schemes.put(TupleScheme.class, new requestSpaceInPlace_resultTupleSchemeFactory()); + schemes.put(StandardScheme.class, new requestSpace_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new requestSpace_resultTupleSchemeFactory()); } public boolean success; // required @@ -12177,13 +9225,13 @@ public String getFieldName() { tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpaceInPlace_result.class, metaDataMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(requestSpace_result.class, metaDataMap); } - public requestSpaceInPlace_result() { + public requestSpace_result() { } - public requestSpaceInPlace_result( + public requestSpace_result( boolean success) { this(); @@ -12194,13 +9242,13 @@ public requestSpaceInPlace_result( /** * Performs a deep copy on other. */ - public requestSpaceInPlace_result(requestSpaceInPlace_result other) { + public requestSpace_result(requestSpace_result other) { __isset_bitfield = other.__isset_bitfield; this.success = other.success; } - public requestSpaceInPlace_result deepCopy() { - return new requestSpaceInPlace_result(this); + public requestSpace_result deepCopy() { + return new requestSpace_result(this); } @Override @@ -12213,7 +9261,7 @@ public boolean isSuccess() { return this.success; } - public requestSpaceInPlace_result setSuccess(boolean success) { + public requestSpace_result setSuccess(boolean success) { this.success = success; setSuccessIsSet(true); return this; @@ -12271,12 +9319,12 @@ public boolean isSet(_Fields field) { public boolean equals(Object that) { if (that == null) return false; - if (that instanceof requestSpaceInPlace_result) - return this.equals((requestSpaceInPlace_result)that); + if (that instanceof requestSpace_result) + return this.equals((requestSpace_result)that); return false; } - public boolean equals(requestSpaceInPlace_result that) { + public boolean equals(requestSpace_result that) { if (that == null) return false; @@ -12298,7 +9346,7 @@ public int hashCode() { } @Override - public int compareTo(requestSpaceInPlace_result other) { + public int compareTo(requestSpace_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } @@ -12332,7 +9380,7 @@ public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache. @Override public String toString() { - StringBuilder sb = new StringBuilder("requestSpaceInPlace_result("); + StringBuilder sb = new StringBuilder("requestSpace_result("); boolean first = true; sb.append("success:"); @@ -12365,15 +9413,15 @@ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException } } - private static class requestSpaceInPlace_resultStandardSchemeFactory implements SchemeFactory { - public requestSpaceInPlace_resultStandardScheme getScheme() { - return new requestSpaceInPlace_resultStandardScheme(); + private static class requestSpace_resultStandardSchemeFactory implements SchemeFactory { + public requestSpace_resultStandardScheme getScheme() { + return new requestSpace_resultStandardScheme(); } } - private static class requestSpaceInPlace_resultStandardScheme extends StandardScheme { + private static class requestSpace_resultStandardScheme extends StandardScheme { - public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpaceInPlace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpace_result struct) throws org.apache.thrift.TException { org.apache.thrift.protocol.TField schemeField; iprot.readStructBegin(); while (true) @@ -12402,7 +9450,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, requestSpaceInPlace struct.validate(); } - public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpaceInPlace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpace_result struct) throws org.apache.thrift.TException { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); @@ -12417,16 +9465,16 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, requestSpaceInPlac } - private static class requestSpaceInPlace_resultTupleSchemeFactory implements SchemeFactory { - public requestSpaceInPlace_resultTupleScheme getScheme() { - return new requestSpaceInPlace_resultTupleScheme(); + private static class requestSpace_resultTupleSchemeFactory implements SchemeFactory { + public requestSpace_resultTupleScheme getScheme() { + return new requestSpace_resultTupleScheme(); } } - private static class requestSpaceInPlace_resultTupleScheme extends TupleScheme { + private static class requestSpace_resultTupleScheme extends TupleScheme { @Override - public void write(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace_result struct) throws org.apache.thrift.TException { + public void write(org.apache.thrift.protocol.TProtocol prot, requestSpace_result struct) throws org.apache.thrift.TException { TTupleProtocol oprot = (TTupleProtocol) prot; BitSet optionals = new BitSet(); if (struct.isSetSuccess()) { @@ -12439,7 +9487,7 @@ public void write(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace } @Override - public void read(org.apache.thrift.protocol.TProtocol prot, requestSpaceInPlace_result struct) throws org.apache.thrift.TException { + public void read(org.apache.thrift.protocol.TProtocol prot, requestSpace_result struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { @@ -12900,7 +9948,6 @@ public void read(org.apache.thrift.protocol.TProtocol prot, unlockBlock_args str public static class unlockBlock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("unlockBlock_result"); - private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -12908,11 +9955,10 @@ public static class unlockBlock_result implements org.apache.thrift.TBase byName = new HashMap(); @@ -12927,8 +9973,6 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { */ public static _Fields findByThriftId(int fieldId) { switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; default: return null; } @@ -12967,15 +10011,9 @@ public String getFieldName() { return _fieldName; } } - - // isset id assignments - private static final int __SUCCESS_ISSET_ID = 0; - private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, - new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(unlockBlock_result.class, metaDataMap); } @@ -12983,20 +10021,10 @@ public String getFieldName() { public unlockBlock_result() { } - public unlockBlock_result( - boolean success) - { - this(); - this.success = success; - setSuccessIsSet(true); - } - /** * Performs a deep copy on other. */ public unlockBlock_result(unlockBlock_result other) { - __isset_bitfield = other.__isset_bitfield; - this.success = other.success; } public unlockBlock_result deepCopy() { @@ -13005,51 +10033,15 @@ public unlockBlock_result deepCopy() { @Override public void clear() { - setSuccessIsSet(false); - this.success = false; - } - - public boolean isSuccess() { - return this.success; - } - - public unlockBlock_result setSuccess(boolean success) { - this.success = success; - setSuccessIsSet(true); - return this; - } - - public void unsetSuccess() { - __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean isSetSuccess() { - return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); - } - - public void setSuccessIsSet(boolean value) { - __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { switch (field) { - case SUCCESS: - if (value == null) { - unsetSuccess(); - } else { - setSuccess((Boolean)value); - } - break; - } } public Object getFieldValue(_Fields field) { switch (field) { - case SUCCESS: - return Boolean.valueOf(isSuccess()); - } throw new IllegalStateException(); } @@ -13061,8 +10053,6 @@ public boolean isSet(_Fields field) { } switch (field) { - case SUCCESS: - return isSetSuccess(); } throw new IllegalStateException(); } @@ -13080,15 +10070,6 @@ public boolean equals(unlockBlock_result that) { if (that == null) return false; - boolean this_present_success = true; - boolean that_present_success = true; - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (this.success != that.success) - return false; - } - return true; } @@ -13105,16 +10086,6 @@ public int compareTo(unlockBlock_result other) { int lastComparison = 0; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); - if (lastComparison != 0) { - return lastComparison; - } - if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); - if (lastComparison != 0) { - return lastComparison; - } - } return 0; } @@ -13135,9 +10106,6 @@ public String toString() { StringBuilder sb = new StringBuilder("unlockBlock_result("); boolean first = true; - sb.append("success:"); - sb.append(this.success); - first = false; sb.append(")"); return sb.toString(); } @@ -13157,8 +10125,6 @@ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOExcept private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bitfield = 0; read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); } catch (org.apache.thrift.TException te) { throw new java.io.IOException(te); @@ -13183,14 +10149,6 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, unlockBlock_result break; } switch (schemeField.id) { - case 0: // SUCCESS - if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { - struct.success = iprot.readBool(); - struct.setSuccessIsSet(true); - } else { - org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); - } - break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -13206,11 +10164,6 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, unlockBlock_result struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - if (struct.isSetSuccess()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); - } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -13228,24 +10181,11 @@ private static class unlockBlock_resultTupleScheme extends TupleScheme String listToString(List list) { return sb.toString(); } - /** - * Write object into ByteBuffer - * - * @param object the object to be written - * @return ByteBuffer contains the object - * @throws IOException - */ - public static ByteBuffer objectToByteBuffer(Object object) throws IOException { - ByteArrayOutputStream bo = new ByteArrayOutputStream(); - ObjectOutputStream oo = new ObjectOutputStream(bo); - oo.writeObject(object); - bo.close(); - oo.close(); - byte[] bytes = bo.toByteArray(); - return ByteBuffer.wrap(bytes); - } - public static String parametersToString(Object... objs) { StringBuilder sb = new StringBuilder("("); for (int k = 0; k < objs.length; k ++) { diff --git a/core/src/main/java/tachyon/worker/BlocksLocker.java b/core/src/main/java/tachyon/worker/BlocksLocker.java index fa90aff82acf..671041620e1c 100644 --- a/core/src/main/java/tachyon/worker/BlocksLocker.java +++ b/core/src/main/java/tachyon/worker/BlocksLocker.java @@ -21,17 +21,13 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import tachyon.StorageDirId; - /** * Handle local block locking. */ public class BlocksLocker { // All Blocks has been locked. private final Map> mLockedBlockIds = new HashMap>(); - // Mapping from block id to id of the StorageDir in which the block is locked - private final Map mLockedBlockIdToStorageDirId = new HashMap(); - // Each user facing block has a unique block lock id. + // Each user facing block has a unique block lock id. private final AtomicInteger mBlockLockId = new AtomicInteger(0); private final int mUserId; @@ -43,45 +39,29 @@ public BlocksLocker(WorkerStorage workerStorage, int userId) { } /** - * Lock a block with lock id. + * Lock a block. * * @param blockId The id of the block. - * @param blockLockId The lock id of the block - * @return The id of the StorageDir in which this block is locked. + * @return The lockId of this lock. */ - public synchronized long lock(long blockId, int blockLockId) { + public synchronized int lock(long blockId) { + int locker = mBlockLockId.incrementAndGet(); if (!mLockedBlockIds.containsKey(blockId)) { - long storageDirIdLocked = mWorkerStorage.lockBlock(blockId, mUserId); - if (!StorageDirId.isUnknown(storageDirIdLocked)) { - mLockedBlockIds.put(blockId, new HashSet()); - mLockedBlockIds.get(blockId).add(blockLockId); - mLockedBlockIdToStorageDirId.put(blockId, storageDirIdLocked); - } - return storageDirIdLocked; - } else { - mLockedBlockIds.get(blockId).add(blockLockId); - return mLockedBlockIdToStorageDirId.get(blockId); + mWorkerStorage.lockBlock(blockId, mUserId); + mLockedBlockIds.put(blockId, new HashSet()); } + mLockedBlockIds.get(blockId).add(locker); + return locker; } - /** - * Get new lock id - */ - public synchronized int getLockId() { - return mBlockLockId.incrementAndGet(); - } /** - * Get id of the StorageDir in which the block is locked + * Check if the block is locked in the local memory * * @param blockId The id of the block - * @return Id of the StorageDir in which the block is locked + * @return true if the block is locked, false otherwise */ - public synchronized long locked(long blockId) { - long storageDirId = StorageDirId.unknownId(); - if (mLockedBlockIds.containsKey(blockId)) { - storageDirId = mLockedBlockIdToStorageDirId.get(blockId); - } - return storageDirId; + public synchronized boolean locked(long blockId) { + return mLockedBlockIds.containsKey(blockId); } /** @@ -89,21 +69,15 @@ public synchronized long locked(long blockId) { * * @param blockId The id of the block. * @param lockId The lock id of the lock. - * @return The id of the StorageDir in which the block is unlocked. */ - public synchronized long unlock(long blockId, int lockId) { + public synchronized void unlock(long blockId, int lockId) { Set lockers = mLockedBlockIds.get(blockId); if (lockers != null) { lockers.remove(lockId); if (lockers.isEmpty()) { mLockedBlockIds.remove(blockId); - mLockedBlockIdToStorageDirId.remove(blockId); - return mWorkerStorage.unlockBlock(blockId, mUserId); - } else { - return mLockedBlockIdToStorageDirId.get(blockId); + mWorkerStorage.unlockBlock(blockId, mUserId); } - } else { - return StorageDirId.unknownId(); } } } diff --git a/core/src/main/java/tachyon/worker/TachyonWorker.java b/core/src/main/java/tachyon/worker/TachyonWorker.java index 94c155d38d78..c0ce5f27ac20 100644 --- a/core/src/main/java/tachyon/worker/TachyonWorker.java +++ b/core/src/main/java/tachyon/worker/TachyonWorker.java @@ -61,13 +61,15 @@ public class TachyonWorker implements Runnable { * @param selectorThreads The number of selector threads of the worker's thrift server * @param acceptQueueSizePerThreads The accept queue size per thread of the worker's thrift server * @param workerThreads The number of threads of the worker's thrift server + * @param localFolder This TachyonWorker's local folder's path + * @param spaceLimitBytes The maximum memory space this TachyonWorker can use, in bytes * @return The new TachyonWorker */ public static synchronized TachyonWorker createWorker(InetSocketAddress masterAddress, InetSocketAddress workerAddress, int dataPort, int selectorThreads, - int acceptQueueSizePerThreads, int workerThreads) { + int acceptQueueSizePerThreads, int workerThreads, String localFolder, long spaceLimitBytes) { return new TachyonWorker(masterAddress, workerAddress, dataPort, selectorThreads, - acceptQueueSizePerThreads, workerThreads); + acceptQueueSizePerThreads, workerThreads, localFolder, spaceLimitBytes); } /** @@ -79,16 +81,19 @@ public static synchronized TachyonWorker createWorker(InetSocketAddress masterAd * @param selectorThreads The number of selector threads of the worker's thrift server * @param acceptQueueSizePerThreads The accept queue size per thread of the worker's thrift server * @param workerThreads The number of threads of the worker's thrift server + * @param localFolder This TachyonWorker's local folder's path + * @param spaceLimitBytes The maximum memory space this TachyonWorker can use, in bytes * @return The new TachyonWorker */ public static synchronized TachyonWorker createWorker(String masterAddress, String workerAddress, - int dataPort, int selectorThreads, int acceptQueueSizePerThreads, int workerThreads) { + int dataPort, int selectorThreads, int acceptQueueSizePerThreads, int workerThreads, + String localFolder, long spaceLimitBytes) { String[] address = masterAddress.split(":"); InetSocketAddress master = new InetSocketAddress(address[0], Integer.parseInt(address[1])); address = workerAddress.split(":"); InetSocketAddress worker = new InetSocketAddress(address[0], Integer.parseInt(address[1])); return new TachyonWorker(master, worker, dataPort, selectorThreads, acceptQueueSizePerThreads, - workerThreads); + workerThreads, localFolder, spaceLimitBytes); } private static String getMasterLocation(String[] args) { @@ -125,7 +130,7 @@ public static void main(String[] args) throws UnknownHostException { TachyonWorker worker = TachyonWorker.createWorker(getMasterLocation(args), resolvedWorkerHost + ":" + wConf.PORT, wConf.DATA_PORT, wConf.SELECTOR_THREADS, wConf.QUEUE_SIZE_PER_SELECTOR, - wConf.SERVER_THREADS); + wConf.SERVER_THREADS, wConf.DATA_FOLDER, wConf.MEMORY_SIZE); try { worker.start(); } catch (Exception e) { @@ -161,9 +166,12 @@ public static void main(String[] args) throws UnknownHostException { * @param selectorThreads The number of selector threads of the worker's thrift server * @param acceptQueueSizePerThreads The accept queue size per thread of the worker's thrift server * @param workerThreads The number of threads of the worker's thrift server + * @param dataFolder This TachyonWorker's local folder's path + * @param memoryCapacityBytes The maximum memory space this TachyonWorker can use, in bytes */ private TachyonWorker(InetSocketAddress masterAddress, InetSocketAddress workerAddress, - int dataPort, int selectorThreads, int acceptQueueSizePerThreads, int workerThreads) { + int dataPort, int selectorThreads, int acceptQueueSizePerThreads, int workerThreads, + String dataFolder, long memoryCapacityBytes) { CommonConf.assertValidPort(masterAddress); CommonConf.assertValidPort(workerAddress); CommonConf.assertValidPort(dataPort); @@ -171,7 +179,7 @@ private TachyonWorker(InetSocketAddress masterAddress, InetSocketAddress workerA mMasterAddress = masterAddress; mWorkerStorage = - new WorkerStorage(mMasterAddress, mExecutorService); + new WorkerStorage(mMasterAddress, dataFolder, memoryCapacityBytes, mExecutorService); mWorkerServiceHandler = new WorkerServiceHandler(mWorkerStorage); @@ -211,9 +219,9 @@ private DataServer createDataServer(final InetSocketAddress dataAddress, final BlocksLocker blockLocker) { switch (WorkerConf.get().NETWORK_TYPE) { case NIO: - return new NIODataServer(dataAddress, blockLocker, mWorkerStorage); + return new NIODataServer(dataAddress, blockLocker); case NETTY: - return new NettyDataServer(dataAddress, blockLocker, mWorkerStorage); + return new NettyDataServer(dataAddress, blockLocker); default: throw new AssertionError("Unknown network type: " + WorkerConf.get().NETWORK_TYPE); } diff --git a/core/src/main/java/tachyon/worker/WorkerClient.java b/core/src/main/java/tachyon/worker/WorkerClient.java index 451dce20fa5e..0b9a1f7cdc25 100644 --- a/core/src/main/java/tachyon/worker/WorkerClient.java +++ b/core/src/main/java/tachyon/worker/WorkerClient.java @@ -41,10 +41,8 @@ import tachyon.thrift.FileDoesNotExistException; import tachyon.thrift.NetAddress; import tachyon.thrift.NoWorkerException; -import tachyon.thrift.OutOfSpaceException; import tachyon.thrift.SuspectedFileSizeException; import tachyon.thrift.TachyonException; -import tachyon.thrift.ClientLocationInfo; import tachyon.thrift.WorkerService; import tachyon.util.NetworkUtils; @@ -61,9 +59,9 @@ public class WorkerClient implements Closeable { private WorkerService.Client mClient; private TProtocol mProtocol; private InetSocketAddress mWorkerAddress; - private NetAddress mWorkerNetAddress; private boolean mConnected = false; private boolean mIsLocal = false; + private String mDataFolder = null; private final ExecutorService mExecutorService; private Future mHeartbeat; @@ -83,14 +81,13 @@ public WorkerClient(MasterClient masterClient, ExecutorService executorService) /** * Update the latest block access time on the worker. * - * @param storageDirId The id of the StorageDir which contains block * @param blockId The id of the block * @throws IOException */ - public synchronized void accessBlock(long storageDirId, long blockId) throws IOException { + public synchronized void accessBlock(long blockId) throws IOException { if (connect()) { try { - mClient.accessBlock(storageDirId, blockId); + mClient.accessBlock(blockId); } catch (TException e) { LOG.error("TachyonClient accessLocalBlock(" + blockId + ") failed"); mConnected = false; @@ -148,15 +145,14 @@ public synchronized boolean asyncCheckpoint(int fid) throws IOException { /** * Notify the worker the block is cached. * - * @param storageDirId The id of StorageDir that the block is stored in * @param blockId The id of the block * @throws IOException */ - public synchronized void cacheBlock(long storageDirId, long blockId) throws IOException { + public synchronized void cacheBlock(long blockId) throws IOException { mustConnect(); try { - mClient.cacheBlock(mMasterClient.getUserId(), storageDirId, blockId); + mClient.cacheBlock(mMasterClient.getUserId(), blockId); } catch (FileDoesNotExistException e) { throw new IOException(e); } catch (BlockInfoException e) { @@ -223,7 +219,6 @@ private synchronized boolean connect() throws IOException { return false; } - mWorkerNetAddress = workerNetAddress; mWorkerAddress = new InetSocketAddress(NetworkUtils.getFqdnHost(workerNetAddress), workerNetAddress.mPort); LOG.info("Connecting " + (mIsLocal ? "local" : "remote") + " worker @ " + mWorkerAddress); @@ -260,45 +255,36 @@ public synchronized InetSocketAddress getAddress() { } /** - * Get location information of the block from the worker - * - * @param blockId The id of the block - * @return the location information of the block + * @return The root local data folder of the worker * @throws IOException */ - public synchronized ClientLocationInfo getLocalBlockLocation(long blockId) - throws IOException { - mustConnect(); + public synchronized String getDataFolder() throws IOException { + if (mDataFolder == null) { + mustConnect(); - try { - return mClient.getLocalBlockLocation(blockId); - } catch (TException e) { - mConnected = false; - throw new IOException(e); + try { + mDataFolder = mClient.getDataFolder(); + } catch (TException e) { + mDataFolder = null; + mConnected = false; + throw new IOException(e); + } } - } - /** - * Get NetAddress of the worker - * - * @return the NetAddress of the worker - */ - public synchronized NetAddress getNetAddress() { - return mWorkerNetAddress; + return mDataFolder; } /** - * Get the location information of the user temporary folder in the StorageDir + * Get the local user temporary folder of the specified user. * - * @param the id of the StorageDir - * @return the path of the user temporary folder in the StorageDir + * @return The local user temporary folder of the specified user + * @throws IOException */ - public synchronized String getUserLocalTempFolder(long storageDirId) - throws IOException { + public synchronized String getUserTempFolder() throws IOException { mustConnect(); try { - return mClient.getUserLocalTempFolder(mMasterClient.getUserId(), storageDirId); + return mClient.getUserTempFolder(mMasterClient.getUserId()); } catch (TException e) { mConnected = false; throw new IOException(e); @@ -345,22 +331,18 @@ public synchronized boolean isLocal() { } /** - * Lock the block, therefore, the worker will not evict the block from the memory until it is + * Lock the block, therefore, the worker will lock evict the block from the memory untill it is * unlocked. * * @param blockId The id of the block * @param userId The id of the user who wants to lock the block - * @return the location information of the StorageDir in which the block is locked * @throws IOException */ - public synchronized ClientLocationInfo lockBlock(long blockId, long userId) - throws IOException { + public synchronized void lockBlock(long blockId, long userId) throws IOException { mustConnect(); try { - return mClient.lockBlock(blockId, userId); - } catch (FileDoesNotExistException e) { - return null; + mClient.lockBlock(blockId, userId); } catch (TException e) { mConnected = false; throw new IOException(e); @@ -383,61 +365,18 @@ public synchronized void mustConnect() throws IOException { } /** - * Promote block back to the top StorageTier - * - * @param userId The id of the user who wants to promote block - * @param blockId The id of the block that will be promoted - * @throws IOException - */ - public synchronized boolean promoteBlock(long userId, long blockId) - throws IOException { - mustConnect(); - - try { - return mClient.promoteBlock(userId, blockId); - } catch (TException e) { - mConnected = false; - throw new IOException(e); - } - } - - /** - * Request space from the worker - * - * @param userId The id of the user who send the request - * @param requestBytes The requested space size, in bytes - * @return the location information of The StorageDir allocated - * @throws IOException - */ - public synchronized ClientLocationInfo requestSpace(long userId, long requestBytes) - throws IOException { - mustConnect(); - - try { - return mClient.requestSpace(userId, requestBytes); - } catch (OutOfSpaceException e) { - return null; - } catch (TException e) { - mConnected = false; - throw new IOException(e); - } - } - - /** - * Request space from the worker in specified StorageDir + * Request space from the worker's memory * * @param userId The id of the user who send the request - * @param storageDirId The id of StorageDir that space will be allocated in * @param requestBytes The requested space size, in bytes * @return true if succeed, false otherwise * @throws IOException */ - public synchronized boolean requestSpace(long userId, long storageDirId, long requestBytes) - throws IOException { + public synchronized boolean requestSpace(long userId, long requestBytes) throws IOException { mustConnect(); try { - return mClient.requestSpaceInPlace(userId, storageDirId, requestBytes); + return mClient.requestSpace(userId, requestBytes); } catch (TException e) { mConnected = false; throw new IOException(e); @@ -448,16 +387,14 @@ public synchronized boolean requestSpace(long userId, long storageDirId, long re * Return the space which has been requested * * @param userId The id of the user who wants to return the space - * @param storageDirId The Id of the StorageDir that space will be returned * @param returnSpaceBytes The returned space size, in bytes * @throws IOException */ - public synchronized void returnSpace(long userId, long storageDirId, long returnSpaceBytes) - throws IOException { + public synchronized void returnSpace(long userId, long returnSpaceBytes) throws IOException { mustConnect(); try { - mClient.returnSpace(userId, storageDirId, returnSpaceBytes); + mClient.returnSpace(userId, returnSpaceBytes); } catch (TException e) { mConnected = false; throw new IOException(e); @@ -469,15 +406,13 @@ public synchronized void returnSpace(long userId, long storageDirId, long return * * @param blockId The id of the block * @param userId The id of the user who wants to unlock the block - * @return true if success, false otherwise * @throws IOException */ - public synchronized boolean unlockBlock(long blockId, long userId) - throws IOException { + public synchronized void unlockBlock(long blockId, long userId) throws IOException { mustConnect(); try { - return mClient.unlockBlock(blockId, userId); + mClient.unlockBlock(blockId, userId); } catch (TException e) { mConnected = false; throw new IOException(e); diff --git a/core/src/main/java/tachyon/worker/WorkerServiceHandler.java b/core/src/main/java/tachyon/worker/WorkerServiceHandler.java index a728c10866fe..677f5431a9bb 100644 --- a/core/src/main/java/tachyon/worker/WorkerServiceHandler.java +++ b/core/src/main/java/tachyon/worker/WorkerServiceHandler.java @@ -19,16 +19,12 @@ import org.apache.thrift.TException; -import tachyon.StorageDirId; import tachyon.thrift.BlockInfoException; import tachyon.thrift.FailedToCheckpointException; import tachyon.thrift.FileDoesNotExistException; -import tachyon.thrift.OutOfSpaceException; import tachyon.thrift.SuspectedFileSizeException; import tachyon.thrift.TachyonException; -import tachyon.thrift.ClientLocationInfo; import tachyon.thrift.WorkerService; -import tachyon.worker.hierarchy.StorageDir; /** * WorkerServiceHandler handles all the RPC calls to the worker. @@ -41,8 +37,8 @@ public WorkerServiceHandler(WorkerStorage workerStorage) { } @Override - public void accessBlock(long storageDirId, long blockId) throws TException { - mWorkerStorage.accessBlock(storageDirId, blockId); + public void accessBlock(long blockId) throws TException { + mWorkerStorage.accessBlock(blockId); } @Override @@ -65,30 +61,23 @@ public boolean asyncCheckpoint(int fileId) throws TachyonException, TException { } @Override - public void cacheBlock(long userId, long storageDirId, long blockId) - throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, TException { + public void cacheBlock(long userId, long blockId) throws FileDoesNotExistException, + SuspectedFileSizeException, BlockInfoException, TException { try { - mWorkerStorage.cacheBlock(userId, storageDirId, blockId); + mWorkerStorage.cacheBlock(userId, blockId); } catch (IOException e) { throw new TException(e); } } @Override - public ClientLocationInfo getLocalBlockLocation(long blockId) - throws FileDoesNotExistException, TException { - StorageDir storageDir = mWorkerStorage.getStorageDirByBlockId(blockId); - if (storageDir == null) { - throw new FileDoesNotExistException("Block not found! blockId:" + blockId); - } else { - return new ClientLocationInfo(storageDir.getStorageDirId(), - storageDir.getBlockFilePath(blockId)); - } + public String getDataFolder() throws TException { + return mWorkerStorage.getDataFolder(); } @Override - public String getUserLocalTempFolder(long userId, long storageDirId) throws TException { - return mWorkerStorage.getUserLocalTempFolder(userId, storageDirId); + public String getUserTempFolder(long userId) throws TException { + return mWorkerStorage.getUserLocalTempFolder(userId); } @Override @@ -97,54 +86,23 @@ public String getUserUfsTempFolder(long userId) throws TException { } @Override - public ClientLocationInfo lockBlock(long blockId, long userId) - throws FileDoesNotExistException, TException { - long storageDirId = mWorkerStorage.lockBlock(blockId, userId); - StorageDir storageDir = mWorkerStorage.getStorageDirById(storageDirId); - if (storageDir == null) { - throw new FileDoesNotExistException("Block file not found! blockId" + blockId); - } else { - return new ClientLocationInfo(storageDir.getStorageDirId(), - storageDir.getBlockFilePath(blockId)); - } - } - - @Override - public boolean promoteBlock(long userId, long blockId) throws TException { - return mWorkerStorage.promoteBlock(userId, blockId); + public void lockBlock(long blockId, long userId) throws TException { + mWorkerStorage.lockBlock(blockId, userId); } @Override - public ClientLocationInfo requestSpace(long userId, long requestBytes) - throws OutOfSpaceException, TException { - StorageDir storageDir = mWorkerStorage.requestSpace(userId, requestBytes); - if (storageDir == null) { - throw new OutOfSpaceException("Failed to allocate space! requestBytes:" + requestBytes); - } else { - return new ClientLocationInfo(storageDir.getStorageDirId(), - storageDir.getUserTempPath(userId)); - } + public boolean requestSpace(long userId, long requestBytes) throws TException { + return mWorkerStorage.requestSpace(userId, requestBytes); } @Override - public boolean requestSpaceInPlace(long userId, long storageDirId, long requestBytes) - throws TException { - return mWorkerStorage.requestSpace(userId, storageDirId, requestBytes); + public void returnSpace(long userId, long returnedBytes) throws TException { + mWorkerStorage.returnSpace(userId, returnedBytes); } @Override - public void returnSpace(long userId, long storageDirId, long returnedBytes) throws TException { - mWorkerStorage.returnSpace(userId, storageDirId, returnedBytes); - } - - @Override - public boolean unlockBlock(long blockId, long userId) throws TException { - long storageDirId = mWorkerStorage.unlockBlock(blockId, userId); - if (StorageDirId.isUnknown(storageDirId)) { - return false; - } else { - return true; - } + public void unlockBlock(long blockId, long userId) throws TException { + mWorkerStorage.unlockBlock(blockId, userId); } @Override diff --git a/core/src/main/java/tachyon/worker/WorkerStorage.java b/core/src/main/java/tachyon/worker/WorkerStorage.java index 6cc8bbb7f30b..778915a7fcc7 100644 --- a/core/src/main/java/tachyon/worker/WorkerStorage.java +++ b/core/src/main/java/tachyon/worker/WorkerStorage.java @@ -15,10 +15,15 @@ package tachyon.worker; +import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; +import java.io.RandomAccessFile; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel.MapMode; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -32,6 +37,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,12 +45,11 @@ import com.google.common.io.Closer; import tachyon.Constants; -import tachyon.StorageDirId; -import tachyon.StorageLevelAlias; import tachyon.UnderFileSystem; import tachyon.Users; import tachyon.conf.CommonConf; import tachyon.conf.WorkerConf; +import tachyon.master.BlockInfo; import tachyon.master.MasterClient; import tachyon.thrift.BlockInfoException; import tachyon.thrift.ClientFileInfo; @@ -55,8 +60,6 @@ import tachyon.thrift.SuspectedFileSizeException; import tachyon.util.CommonUtils; import tachyon.util.ThreadFactoryUtils; -import tachyon.worker.hierarchy.StorageDir; -import tachyon.worker.hierarchy.StorageTier; /** * The structure to store a worker's information in worker node. @@ -183,8 +186,8 @@ public void run() { // master String midPath = CommonUtils.concat(mUfsWorkerDataFolder, fileId); String dstPath = CommonUtils.concat(CommonConf.get().UNDERFS_DATA_FOLDER, fileId); - LOG.info("Thread " + mId + " is checkpointing file " + fileId + " to " + midPath + " to " - + dstPath); + LOG.info("Thread " + mId + " is checkpointing file " + fileId + " from " + + mLocalDataFolder.toString() + " to " + midPath + " to " + dstPath); if (mCheckpointUfs == null) { mCheckpointUfs = UnderFileSystem.get(midPath); @@ -196,42 +199,31 @@ public void run() { LOG.error("File " + fileInfo + " is not complete!"); continue; } - - long[] storageDirIds = new long[fileInfo.blockIds.size()]; + for (int k = 0; k < fileInfo.blockIds.size(); k ++) { + lockBlock(fileInfo.blockIds.get(k), Users.CHECKPOINT_USER_ID); + } Closer closer = Closer.create(); long fileSizeByte = 0; try { - for (int k = 0; k < fileInfo.blockIds.size(); k ++) { - long blockId = fileInfo.blockIds.get(k); - storageDirIds[k] = lockBlock(blockId, Users.CHECKPOINT_USER_ID); - if (StorageDirId.isUnknown(storageDirIds[k])) { - throw new IOException("Block doesn't exist!"); - } - } OutputStream os = closer.register(mCheckpointUfs.create(midPath, (int) fileInfo.getBlockSizeByte())); for (int k = 0; k < fileInfo.blockIds.size(); k ++) { - StorageDir storageDir = getStorageDirById(storageDirIds[k]); - BlockHandler handler = - closer.register(storageDir.getBlockHandler(fileInfo.blockIds.get(k))); - ByteBuffer byteBuffer = handler.read(0, -1); + File tempFile = + new File(CommonUtils.concat(mLocalDataFolder.toString(), + fileInfo.blockIds.get(k))); + fileSizeByte += tempFile.length(); + InputStream is = closer.register(new FileInputStream(tempFile)); byte[] buf = new byte[16 * Constants.KB]; - int writeLen; - while (byteBuffer.remaining() > 0) { - if (byteBuffer.remaining() >= buf.length) { - writeLen = buf.length; - } else { - writeLen = byteBuffer.remaining(); - } - byteBuffer.get(buf, 0, writeLen); - os.write(buf, 0, writeLen); + int got = is.read(buf); + while (got != -1) { + os.write(buf, 0, got); + got = is.read(buf); } } } finally { closer.close(); for (int k = 0; k < fileInfo.blockIds.size(); k ++) { - long blockId = fileInfo.blockIds.get(k); - unlockBlock(blockId, Users.CHECKPOINT_USER_ID); + unlockBlock(fileInfo.blockIds.get(k), Users.CHECKPOINT_USER_ID); } } if (!mCheckpointUfs.rename(midPath, dstPath)) { @@ -261,11 +253,23 @@ public void run() { private volatile MasterClient mMasterClient; private final InetSocketAddress mMasterAddress; private NetAddress mWorkerAddress; + private final SpaceCounter mSpaceCounter; private long mWorkerId; + private final Set mMemoryData = new HashSet(); + private final Map mBlockSizes = new HashMap(); - private final String mDataFolder; - private final String mUserFolder; + private final Map mBlockIdToLatestAccessTimeMs = new HashMap(); + private final Map> mLockedBlockIdToUserId = new HashMap>(); + + private final Map> mLockedBlocksPerUser = new HashMap>(); + private final BlockingQueue mRemovedBlockList = new ArrayBlockingQueue( + Constants.WORKER_BLOCKS_QUEUE_SIZE); + + private final BlockingQueue mAddedBlockList = new ArrayBlockingQueue( + Constants.WORKER_BLOCKS_QUEUE_SIZE); + private final File mLocalDataFolder; + private final File mLocalUserFolder; private String mUfsWorkerFolder; private String mUfsWorkerDataFolder; private String mUfsOrphansFolder; @@ -286,10 +290,6 @@ public void run() { ThreadFactoryUtils.build("checkpoint-%d")); private final ExecutorService mExecutorService; - private long mCapacityBytes; - private StorageTier[] mStorageTiers; - private final BlockingQueue mRemovedBlockIdList = new ArrayBlockingQueue( - Constants.WORKER_BLOCKS_QUEUE_SIZE); /** * Main logic behind the worker process. @@ -298,43 +298,43 @@ public void run() { * {@link #initialize} must be called. * * @param masterAddress The TachyonMaster's address + * @param dataFolder This TachyonWorker's local folder's path + * @param memoryCapacityBytes The maximum memory space this TachyonWorker can use, in bytes * @param executorService */ - public WorkerStorage(InetSocketAddress masterAddress, ExecutorService executorService) { + public WorkerStorage(InetSocketAddress masterAddress, String dataFolder, + long memoryCapacityBytes, ExecutorService executorService) { mExecutorService = executorService; mCommonConf = CommonConf.get(); mMasterAddress = masterAddress; mMasterClient = new MasterClient(mMasterAddress, mExecutorService); + mLocalDataFolder = new File(dataFolder); - mDataFolder = WorkerConf.get().DATA_FOLDER; - mUserFolder = CommonUtils.concat(mDataFolder, WorkerConf.USER_TEMP_RELATIVE_FOLDER); + mSpaceCounter = new SpaceCounter(memoryCapacityBytes); + mLocalUserFolder = new File(mLocalDataFolder, WorkerConf.USER_TEMP_RELATIVE_FOLDER); } public void initialize(final NetAddress address) { mWorkerAddress = address; - try { - initializeStorageTier(); - } catch (IOException e) { - throw Throwables.propagate(e); - } - register(); mUfsWorkerFolder = CommonUtils.concat(mCommonConf.UNDERFS_WORKERS_FOLDER, mWorkerId); mUfsWorkerDataFolder = mUfsWorkerFolder + "/data"; mUfs = UnderFileSystem.get(mCommonConf.UNDERFS_ADDRESS); - mUsers = new Users(mUserFolder, mUfsWorkerFolder); + mUsers = new Users(mLocalUserFolder.toString(), mUfsWorkerFolder); for (int k = 0; k < WorkerConf.get().WORKER_CHECKPOINT_THREADS; k ++) { mCheckpointExecutor.submit(new CheckpointThread(k)); } try { - addFoundBlocks(); + initializeWorkerStorage(); } catch (IOException e) { throw Throwables.propagate(e); + } catch (FileDoesNotExistException e) { + throw Throwables.propagate(e); } catch (SuspectedFileSizeException e) { throw Throwables.propagate(e); } catch (BlockInfoException e) { @@ -342,19 +342,25 @@ public void initialize(final NetAddress address) { } LOG.info("Current Worker Info: ID " + mWorkerId + ", mWorkerAddress: " + mWorkerAddress - + ", CapacityBytes: " + mCapacityBytes); + + ", MemoryCapacityBytes: " + mSpaceCounter.getCapacityBytes()); } /** * Update the latest block access time on the worker. * - * @param storageDirId The id of the StorageDir which block is in * @param blockId The id of the block */ - void accessBlock(long storageDirId, long blockId) { - StorageDir foundDir = getStorageDirById(storageDirId); - if (foundDir != null) { - foundDir.accessBlock(blockId); + void accessBlock(long blockId) { + synchronized (mBlockIdToLatestAccessTimeMs) { + mBlockIdToLatestAccessTimeMs.put(blockId, System.currentTimeMillis()); + } + } + + private void addBlockId(long blockId, long fileSizeBytes) { + synchronized (mBlockIdToLatestAccessTimeMs) { + mBlockIdToLatestAccessTimeMs.put(blockId, System.currentTimeMillis()); + mBlockSizes.put(blockId, fileSizeBytes); + mMemoryData.add(blockId); } } @@ -395,32 +401,10 @@ public void addCheckpoint(long userId, int fileId) throws FileDoesNotExistExcept mMasterClient.addCheckpoint(mWorkerId, fileId, fileSize, dstPath); } - /** - * Report blocks on the worker when initializing worker storage - * - * @throws IOException - * @throws BlockInfoException - * @throws SuspectedFileSizeException - */ - private void addFoundBlocks() throws IOException, SuspectedFileSizeException, BlockInfoException { - mUfsOrphansFolder = mUfsWorkerFolder + "/orphans"; - if (!mUfs.exists(mUfsOrphansFolder)) { - mUfs.mkdirs(mUfsOrphansFolder, true); - } - for (StorageTier curStorageTier : mStorageTiers) { - for (StorageDir curStorageDir : curStorageTier.getStorageDirs()) { - for (Entry blockSize : curStorageDir.getBlockSizes()) { - try { - mMasterClient.worker_cacheBlock(mWorkerId, getUsedBytes(), - curStorageDir.getStorageDirId(), blockSize.getKey(), blockSize.getValue()); - } catch (FileDoesNotExistException e) { - LOG.error("BlockId: " + blockSize.getKey() + " Not Exist in Metadata"); - swapoutOrphanBlocks(curStorageDir, blockSize.getKey()); - freeBlock(blockSize.getKey()); - } - } - } - } + private void addFoundBlock(long blockId, long length) throws FileDoesNotExistException, + SuspectedFileSizeException, BlockInfoException, IOException { + addBlockId(blockId, length); + mMasterClient.worker_cacheBlock(mWorkerId, mSpaceCounter.getUsedBytes(), blockId, length); } /** @@ -461,29 +445,34 @@ public boolean asyncCheckpoint(int fileId) throws IOException { * work on local files. * * @param userId The user id of the client who send the notification - * @param storageDirId The id of the StorageDir that block is cached into * @param blockId The id of the block * @throws FileDoesNotExistException * @throws SuspectedFileSizeException * @throws BlockInfoException * @throws IOException */ - public void cacheBlock(long userId, long storageDirId, long blockId) - throws FileDoesNotExistException, SuspectedFileSizeException, BlockInfoException, - IOException { - StorageDir storageDir = getStorageDirById(storageDirId); - if (storageDir != null) { - try { - storageDir.cacheBlock(userId, blockId); - } catch (IOException e) { - throw new FileDoesNotExistException("Failed to cache block! block id:" + blockId); + public void cacheBlock(long userId, long blockId) throws FileDoesNotExistException, + SuspectedFileSizeException, BlockInfoException, IOException { + File srcFile = new File(CommonUtils.concat(getUserLocalTempFolder(userId), blockId)); + File dstFile = new File(CommonUtils.concat(mLocalDataFolder, blockId)); + long fileSizeBytes = srcFile.length(); + if (!srcFile.exists()) { + throw new FileDoesNotExistException("File " + srcFile + " does not exist."); + } + synchronized (mBlockIdToLatestAccessTimeMs) { + if (!srcFile.renameTo(dstFile)) { + throw new FileDoesNotExistException("Failed to rename file from " + srcFile.getPath() + + " to " + dstFile.getPath()); } - long blockSize = storageDir.getBlockSize(blockId); - mUsers.addOwnBytes(userId, blockSize); - mMasterClient.worker_cacheBlock(mWorkerId, getUsedBytes(), storageDirId, blockId, blockSize); - } else { - throw new FileDoesNotExistException("StorageDir doesn't exist! ID:" + storageDirId); + if (mBlockSizes.containsKey(blockId)) { + mSpaceCounter.returnUsedBytes(mBlockSizes.get(blockId)); + } + addBlockId(blockId, fileSizeBytes); + mUsers.addOwnBytes(userId, -fileSizeBytes); + mMasterClient.worker_cacheBlock(mWorkerId, mSpaceCounter.getUsedBytes(), blockId, + fileSizeBytes); } + LOG.info(userId + " " + dstFile); } /** @@ -494,32 +483,44 @@ public void cacheBlock(long userId, long storageDirId, long blockId) public void checkStatus() { List removedUsers = mUsers.checkStatus(); - for (StorageTier curTier : mStorageTiers) { - for (StorageDir curDir : curTier.getStorageDirs()) { - curDir.checkStatus(removedUsers); - } - } - for (long userId : removedUsers) { - mUsers.removeUser(userId); + mSpaceCounter.returnUsedBytes(mUsers.removeUser(userId)); + synchronized (mLockedBlockIdToUserId) { + Set blockds = mLockedBlocksPerUser.get(userId); + mLockedBlocksPerUser.remove(userId); + if (blockds != null) { + for (long blockId : blockds) { + unlockBlock(blockId, userId); + } + } + } } } /** - * Remove a block from WorkerStorage. + * Remove a block from the memory. * * @param blockId The block to be removed. - * @throws IOException + * @return Removed file size in bytes. */ - private void freeBlock(long blockId) throws IOException { - for (StorageTier storageTier : mStorageTiers) { - for (StorageDir storageDir : storageTier.getStorageDirs()) { - if (storageDir.containsBlock(blockId)) { - storageDir.deleteBlock(blockId); - } + private long freeBlock(long blockId) { + long freedFileBytes = 0; + synchronized (mBlockIdToLatestAccessTimeMs) { + if (mBlockSizes.containsKey(blockId)) { + mSpaceCounter.returnUsedBytes(mBlockSizes.get(blockId)); + File srcFile = new File(CommonUtils.concat(mLocalDataFolder, blockId)); + srcFile.delete(); + mBlockIdToLatestAccessTimeMs.remove(blockId); + freedFileBytes = mBlockSizes.remove(blockId); + mRemovedBlockList.add(blockId); + mMemoryData.remove(blockId); + LOG.info("Removed Data " + blockId); + } else { + LOG.warn("File " + blockId + " does not exist in memory."); } } - mRemovedBlockIdList.add(blockId); + + return freedFileBytes; } /** @@ -528,15 +529,11 @@ private void freeBlock(long blockId) throws IOException { * This is triggered when the worker heartbeats to the master, which sends a * {@link tachyon.thrift.Command} with type {@link tachyon.thrift.CommandType#Free} * - * @param blockIds The id list of blocks to be removed. + * @param blocks The list of blocks to be removed. */ - public void freeBlocks(List blockIds) { - for (long blockId : blockIds) { - try { - freeBlock(blockId); - } catch (IOException e) { - LOG.error("Failed to delete block file! blockId:" + blockId); - } + public void freeBlocks(List blocks) { + for (long blockId : blocks) { + freeBlock(blockId); } } @@ -544,42 +541,7 @@ public void freeBlocks(List blockIds) { * @return The root local data folder of the worker */ public String getDataFolder() { - return mDataFolder; - } - - /** - * Get StorageDir which contains specified block - * - * @param blockId the id of the block - * @return StorageDir which contains the block - */ - public StorageDir getStorageDirByBlockId(long blockId) { - StorageDir storageDir = null; - for (StorageTier storageTier : mStorageTiers) { - storageDir = storageTier.getStorageDirByBlockId(blockId); - if (storageDir != null) { - return storageDir; - } - } - return null; - } - - /** - * Get StorageDir specified by id - * - * @param storageDirId the id of the StorageDir - * @return StorageDir specified by the id - */ - public StorageDir getStorageDirById(long storageDirId) { - int storageLevel = StorageDirId.getStorageLevel(storageDirId); - int dirIndex = StorageDirId.getStorageDirIndex(storageDirId); - if (storageLevel >= 0 && storageLevel < mStorageTiers.length) { - StorageDir[] storageDirs = mStorageTiers[storageLevel].getStorageDirs(); - if (dirIndex >= 0 && dirIndex < storageDirs.length) { - return storageDirs[dirIndex]; - } - } - return null; + return mLocalDataFolder.toString(); } /** @@ -589,26 +551,13 @@ public String getUfsOrphansFolder() { return mUfsOrphansFolder; } - /** - * Get used bytes of current WorkerStorage - * - * @return used bytes of current WorkerStorage - */ - private long getUsedBytes() { - long usedBytes = 0; - for (StorageTier curTier : mStorageTiers) { - usedBytes += curTier.getUsedBytes(); - } - return usedBytes; - } - /** * Get the local user temporary folder of the specified user. * * This method is a wrapper around {@link tachyon.Users#getUserTempFolder(long)}, and as such * should be referentially transparent with {@link tachyon.Users#getUserTempFolder(long)}. In the * context of {@code this}, this call will output the result of path concat of - * {@link #mUserFolder} with the provided {@literal userId}. + * {@link #mLocalUserFolder} with the provided {@literal userId}. * * This method differs from {@link #getUserUfsTempFolder(long)} in the context of where write * operations end up. This temp folder generated lives inside the tachyon file system, and as @@ -619,15 +568,10 @@ private long getUsedBytes() { * @param userId The id of the user * @return The local user temporary folder of the specified user */ - public String getUserLocalTempFolder(long userId, long storageDirId) { - StorageDir storageDir = getStorageDirById(storageDirId); - if (storageDir != null) { - String userLocalTempFolder = storageDir.getUserTempPath(userId); - LOG.info("Return UserTempFolder for " + userId + " : " + userLocalTempFolder); - return userLocalTempFolder; - } else { - return ""; - } + public String getUserLocalTempFolder(long userId) { + String ret = mUsers.getUserTempFolder(userId); + LOG.info("Return UserTempFolder for " + userId + " : " + ret); + return ret; } /** @@ -658,56 +602,70 @@ public String getUserUfsTempFolder(long userId) { * @throws IOException */ public Command heartbeat() throws IOException { - List removedBlockIds = new ArrayList(); - Map> addedBlockIds = new HashMap>(); + ArrayList sendRemovedPartitionList = new ArrayList(); + while (mRemovedBlockList.size() > 0) { + sendRemovedPartitionList.add(mRemovedBlockList.poll()); + } + return mMasterClient.worker_heartbeat(mWorkerId, mSpaceCounter.getUsedBytes(), + sendRemovedPartitionList); + } - mRemovedBlockIdList.drainTo(removedBlockIds); + private void initializeWorkerStorage() throws IOException, FileDoesNotExistException, + SuspectedFileSizeException, BlockInfoException { + LOG.info("Initializing the worker storage."); + if (!mLocalDataFolder.exists()) { + LOG.info("Local folder " + mLocalDataFolder + " does not exist. Creating a new one."); + mLocalDataFolder.mkdirs(); + mLocalUserFolder.mkdirs(); - for (StorageTier storageTier : mStorageTiers) { - for (StorageDir storageDir : storageTier.getStorageDirs()) { - addedBlockIds.put(storageDir.getStorageDirId(), storageDir.getAddedBlockIdList()); - } + CommonUtils.changeLocalFilePermission(mLocalDataFolder.getPath(), "775"); + CommonUtils.changeLocalFilePermission(mLocalUserFolder.getPath(), "775"); + return; } - return mMasterClient - .worker_heartbeat(mWorkerId, getUsedBytes(), removedBlockIds, addedBlockIds); - } - /** - * Initialize StorageTiers on current WorkerStorage - * - * @throws IOException - */ - public void initializeStorageTier() throws IOException { - mStorageTiers = new StorageTier[WorkerConf.get().MAX_HIERARCHY_STORAGE_LEVEL]; - StorageTier nextStorageTier = null; - for (int level = mStorageTiers.length - 1; level >= 0; level --) { - if (WorkerConf.get().STORAGE_TIER_DIRS[level] == null) { - throw new IOException("No directory path is set for layer " + level); - } - String[] dirPaths = WorkerConf.get().STORAGE_TIER_DIRS[level].split(","); - for (int i = 0; i < dirPaths.length; i ++) { - dirPaths[i] = dirPaths[i].trim(); - } - StorageLevelAlias alias = WorkerConf.get().STORAGE_LEVEL_ALIAS[level]; - if (WorkerConf.get().STORAGE_TIER_DIR_QUOTA[level] == null) { - throw new IOException("No directory quota is set for layer " + level); + if (!mLocalDataFolder.isDirectory()) { + String tmp = "Data folder " + mLocalDataFolder + " is not a folder!"; + LOG.error(tmp); + throw new IllegalArgumentException(tmp); + } + + if (mLocalUserFolder.exists()) { + try { + FileUtils.deleteDirectory(mLocalUserFolder); + } catch (IOException e) { + LOG.error(e.getMessage(), e); } - String[] dirCapacityStrings = WorkerConf.get().STORAGE_TIER_DIR_QUOTA[level].split(","); - long[] dirCapacities = new long[dirPaths.length]; - for (int i = 0, j = 0; i < dirPaths.length; i ++) { - // The storage directory quota for each storage directory - dirCapacities[i] = CommonUtils.parseSpaceSize(dirCapacityStrings[j].trim()); - if (j < dirCapacityStrings.length - 1) { - j ++; + } + mLocalUserFolder.mkdir(); + CommonUtils.changeLocalFilePermission(mLocalUserFolder.getPath(), "775"); + + mUfsOrphansFolder = mUfsWorkerFolder + "/orphans"; + if (!mUfs.exists(mUfsOrphansFolder)) { + mUfs.mkdirs(mUfsOrphansFolder, true); + } + + int cnt = 0; + for (File tFile : mLocalDataFolder.listFiles()) { + if (tFile.isFile()) { + cnt ++; + LOG.info("File " + cnt + ": " + tFile.getPath() + " with size " + tFile.length() + " Bs."); + + long blockId = CommonUtils.getBlockIdFromFileName(tFile.getName()); + boolean success = mSpaceCounter.requestSpaceBytes(tFile.length()); + try { + addFoundBlock(blockId, tFile.length()); + } catch (FileDoesNotExistException e) { + LOG.error("BlockId: " + blockId + " becomes orphan for: \"" + e.message + "\""); + LOG.info("Swapout File " + cnt + ": blockId: " + blockId + " to " + mUfsOrphansFolder); + swapoutOrphanBlocks(blockId, tFile); + freeBlock(blockId); + continue; + } + mAddedBlockList.add(blockId); + if (!success) { + throw new RuntimeException("Pre-existing files exceed the local memory capacity."); } } - StorageTier curStorageTier = - new StorageTier(level, alias, dirPaths, dirCapacities, mDataFolder, mUserFolder, - nextStorageTier, null); // TODO add conf for UFS - curStorageTier.initialize(); - mCapacityBytes += curStorageTier.getCapacityBytes(); - mStorageTiers[level] = curStorageTier; - nextStorageTier = curStorageTier; } } @@ -722,59 +680,61 @@ public void initializeStorageTier() throws IOException { * * @param blockId The id of the block * @param userId The id of the user who locks the block - * @return the Id of the StorageDir in which the block is locked */ - public long lockBlock(long blockId, long userId) { - StorageDir storageDir = getStorageDirByBlockId(blockId); - if (storageDir != null) { - if (storageDir.lockBlock(blockId, userId)) { - return storageDir.getStorageDirId(); + public void lockBlock(long blockId, long userId) { + synchronized (mLockedBlockIdToUserId) { + if (!mLockedBlockIdToUserId.containsKey(blockId)) { + mLockedBlockIdToUserId.put(blockId, new HashSet()); } + mLockedBlockIdToUserId.get(blockId).add(userId); + + if (!mLockedBlocksPerUser.containsKey(userId)) { + mLockedBlocksPerUser.put(userId, new HashSet()); + } + mLockedBlocksPerUser.get(userId).add(blockId); } - LOG.warn(String.format("Failed to lock block! blockId(%d)", blockId)); - return StorageDirId.unknownId(); } /** - * Promote block back to top StorageTier + * Use local LRU to evict data, and get requestBytes available space. * - * @param userId the id of the user - * @param blockId the id of the block - * @return true if success, false otherwise + * @param requestBytes The data requested. + * @return true if the space is granted, false if not. */ - public boolean promoteBlock(long userId, long blockId) { - long storageDirIdLocked = lockBlock(blockId, userId); - if (StorageDirId.isUnknown(storageDirIdLocked)) { - return false; - } else if (StorageDirId.getStorageLevelAliasValue(storageDirIdLocked) != mStorageTiers[0] - .getStorageLevelAlias().getValue()) { - StorageDir srcStorageDir = getStorageDirById(storageDirIdLocked); - long blockSize = srcStorageDir.getBlockSize(blockId); - StorageDir dstStorageDir = requestSpace(userId, blockSize); - if (dstStorageDir == null) { - LOG.error("Failed to promote block! blockId:" + blockId); - srcStorageDir.unlockBlock(blockId, userId); - return false; - } - boolean result = false; - try { - try { - result = srcStorageDir.copyBlock(blockId, dstStorageDir); - } finally { - srcStorageDir.unlockBlock(blockId, userId); - } - if (result) { - srcStorageDir.deleteBlock(blockId); + private boolean memoryEvictionLRU(long requestBytes) { + Set pinList; + + try { + pinList = mMasterClient.worker_getPinIdList(); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + pinList = new HashSet(); + } + + synchronized (mBlockIdToLatestAccessTimeMs) { + synchronized (mLockedBlockIdToUserId) { + while (mSpaceCounter.getAvailableBytes() < requestBytes) { + long blockId = -1; + long latestTimeMs = Long.MAX_VALUE; + for (Entry entry : mBlockIdToLatestAccessTimeMs.entrySet()) { + if (entry.getValue() < latestTimeMs + && !pinList.contains(BlockInfo.computeInodeId(entry.getKey()))) { + if (!mLockedBlockIdToUserId.containsKey(entry.getKey())) { + blockId = entry.getKey(); + latestTimeMs = entry.getValue(); + } + } + } + if (blockId != -1) { + freeBlock(blockId); + } else { + return false; + } } - return result; - } catch (IOException e) { - LOG.error("Failed to promote block! blockId:" + blockId); - return false; } - } else { - unlockBlock(blockId, userId); - return true; } + + return true; } /** @@ -782,19 +742,11 @@ public boolean promoteBlock(long userId, long blockId) { */ public void register() { long id = 0; - Map> blockIdLists = new HashMap>(); - - for (StorageTier curStorageTier : mStorageTiers) { - for (StorageDir curStorageDir : curStorageTier.getStorageDirs()) { - Set blockSet = curStorageDir.getBlockIds(); - blockIdLists.put(curStorageDir.getStorageDirId(), new ArrayList(blockSet)); - } - } while (id == 0) { try { id = - mMasterClient.worker_register(mWorkerAddress, mCapacityBytes, getUsedBytes(), - blockIdLists); + mMasterClient.worker_register(mWorkerAddress, mSpaceCounter.getCapacityBytes(), + mSpaceCounter.getUsedBytes(), new ArrayList(mMemoryData)); } catch (BlockInfoException e) { LOG.error(e.getMessage(), e); id = 0; @@ -813,75 +765,26 @@ public void register() { * * @param userId The id of the user who send the request * @param requestBytes The requested space size, in bytes - * @return StorageDir assigned if succeed, null otherwise - */ - public StorageDir requestSpace(long userId, long requestBytes) { - Set pinList; - - try { - pinList = mMasterClient.worker_getPinIdList(); - } catch (IOException e) { - LOG.error(e.getMessage()); - pinList = new HashSet(); - } - - StorageDir storageDir; - List removedBlockIds = new ArrayList(); - try { - storageDir = mStorageTiers[0].requestSpace(userId, requestBytes, pinList, removedBlockIds); - } catch (IOException e) { - LOG.error(e.getMessage()); - storageDir = null; - } finally { - if (removedBlockIds.size() > 0) { - mRemovedBlockIdList.addAll(removedBlockIds); - } - } - - if (storageDir != null) { - mUsers.addOwnBytes(userId, requestBytes); - } - return storageDir; - } - - /** - * Request space from the specified StorageDir - * - * @param userId The id of the user who send the request - * @param storageDirId The id of the StorageDir specified - * @param requestBytes The requested space size, in bytes * @return true if succeed, false otherwise */ - public boolean requestSpace(long userId, long storageDirId, long requestBytes) { - Set pinList; - - try { - pinList = mMasterClient.worker_getPinIdList(); - } catch (IOException e) { - LOG.error(e.getMessage()); - pinList = new HashSet(); + public boolean requestSpace(long userId, long requestBytes) { + LOG.info("requestSpace(" + userId + ", " + requestBytes + "): Current available: " + + mSpaceCounter.getAvailableBytes() + " requested: " + requestBytes); + if (mSpaceCounter.getCapacityBytes() < requestBytes) { + LOG.info("user_requestSpace(): requested memory size is larger than the total memory on" + + " the machine."); + return false; } - StorageDir storageDir = getStorageDirById(storageDirId); - boolean result; - List removedBlockIds = new ArrayList(); - try { - result = - mStorageTiers[0].requestSpace(storageDir, userId, requestBytes, pinList, - removedBlockIds); - } catch (IOException e) { - LOG.error(e.getMessage()); - result = false; - } finally { - if (removedBlockIds.size() > 0) { - mRemovedBlockIdList.addAll(removedBlockIds); + while (!mSpaceCounter.requestSpaceBytes(requestBytes)) { + if (!memoryEvictionLRU(requestBytes)) { + return false; } } - if (result) { - mUsers.addOwnBytes(userId, requestBytes); - } - return result; + mUsers.addOwnBytes(userId, requestBytes); + + return true; } /** @@ -896,24 +799,19 @@ public void resetMasterClient() { * Return the space which has been requested * * @param userId The id of the user who wants to return the space - * @param storageDirId The id of the StorageDir space will be returned to * @param returnedBytes The returned space size, in bytes */ - public void returnSpace(long userId, long storageDirId, long returnedBytes) { - StorageDir storageDir = getStorageDirById(storageDirId); - if (storageDir == null) { - LOG.warn("StorageDir doesn't exist! ID:" + storageDirId); - return; - } + public void returnSpace(long userId, long returnedBytes) { + long preAvailableBytes = mSpaceCounter.getAvailableBytes(); if (returnedBytes > mUsers.ownBytes(userId)) { LOG.error("User " + userId + " does not own " + returnedBytes + " bytes."); } else { - storageDir.returnSpace(userId, returnedBytes); + mSpaceCounter.returnUsedBytes(returnedBytes); mUsers.addOwnBytes(userId, -returnedBytes); } - LOG.info("returnSpace(" + userId + ", " + returnedBytes + ") : " + " New Available: " - + storageDir.getAvailableBytes()); + LOG.info("returnSpace(" + userId + ", " + returnedBytes + ") : " + preAvailableBytes + + " returned: " + returnedBytes + ". New Available: " + mSpaceCounter.getAvailableBytes()); } /** @@ -936,8 +834,10 @@ public void stop() { * Swap out those blocks missing INode information onto underFS which can be retrieved by user * later. Its cleanup only happens while formating the mTachyonFS. */ - private void swapoutOrphanBlocks(StorageDir storageDir, long blockId) throws IOException { - ByteBuffer buf = storageDir.getBlockData(blockId, 0, -1); + private void swapoutOrphanBlocks(long blockId, File file) throws IOException { + RandomAccessFile localFile = new RandomAccessFile(file, "r"); + ByteBuffer buf = localFile.getChannel().map(MapMode.READ_ONLY, 0, file.length()); + String ufsOrphanBlock = CommonUtils.concat(mUfsOrphansFolder, blockId); OutputStream os = mUfs.create(ufsOrphanBlock); final int bulkSize = Constants.KB * 64; @@ -948,6 +848,8 @@ private void swapoutOrphanBlocks(StorageDir storageDir, long blockId) throws IOE os.write(bulk, 0, len); } os.close(); + + localFile.close(); } /** @@ -961,17 +863,20 @@ private void swapoutOrphanBlocks(StorageDir storageDir, long blockId) throws IOE * * @param blockId The id of the block * @param userId The id of the user who unlocks the block - * @return the Id of the StorageDir in which the block is unlocked */ - public long unlockBlock(long blockId, long userId) { - StorageDir storageDir = getStorageDirByBlockId(blockId); - if (storageDir != null) { - if (storageDir.unlockBlock(blockId, userId)) { - return storageDir.getStorageDirId(); + public void unlockBlock(long blockId, long userId) { + synchronized (mLockedBlockIdToUserId) { + if (mLockedBlockIdToUserId.containsKey(blockId)) { + mLockedBlockIdToUserId.get(blockId).remove(userId); + if (mLockedBlockIdToUserId.get(blockId).size() == 0) { + mLockedBlockIdToUserId.remove(blockId); + } + } + + if (mLockedBlocksPerUser.containsKey(userId)) { + mLockedBlocksPerUser.get(userId).remove(blockId); } } - LOG.warn(String.format("Failed to unlock block! blockId(%d)", blockId)); - return StorageDirId.unknownId(); } /** diff --git a/core/src/main/java/tachyon/worker/hierarchy/StorageDir.java b/core/src/main/java/tachyon/worker/hierarchy/StorageDir.java index 2970defe3776..6ee96dc4877f 100644 --- a/core/src/main/java/tachyon/worker/hierarchy/StorageDir.java +++ b/core/src/main/java/tachyon/worker/hierarchy/StorageDir.java @@ -19,8 +19,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map.Entry; import java.util.Set; @@ -40,9 +38,8 @@ import tachyon.Constants; import tachyon.TachyonURI; import tachyon.UnderFileSystem; -import tachyon.Users; +import tachyon.client.BlockHandler; import tachyon.util.CommonUtils; -import tachyon.worker.BlockHandler; import tachyon.worker.SpaceCounter; /** @@ -55,11 +52,9 @@ public final class StorageDir { /** Mapping from blockId to its last access time in milliseconds */ private final ConcurrentMap mLastBlockAccessTimeMs = new ConcurrentHashMap(); - /** List of added block Ids to be reported */ - private final BlockingQueue mAddedBlockIdList = new ArrayBlockingQueue( + /** List of removed block Ids */ + private final BlockingQueue mRemovedBlockIdList = new ArrayBlockingQueue( Constants.WORKER_BLOCKS_QUEUE_SIZE); - /** List of to be removed block Ids */ - private final Set mToRemoveBlockIdSet = Collections.synchronizedSet(new HashSet()); /** Space counter of current StorageDir */ private final SpaceCounter mSpaceCounter; /** Id of StorageDir */ @@ -74,8 +69,8 @@ public final class StorageDir { private final UnderFileSystem mFs; /** Configuration of under file system */ private final Object mConf; - /** Mapping from user Id to available space in bytes */ - private final ConcurrentMap mUserAvailableBytes = new ConcurrentHashMap(); + /** Mapping from user Id to allocated space in bytes */ + private final ConcurrentMap mUserAllocatedBytes = new ConcurrentHashMap(); /** Mapping from user Id to list of blocks locked by the user */ private final Multimap mLockedBlocksPerUser = Multimaps .synchronizedMultimap(HashMultimap.create()); @@ -110,9 +105,7 @@ public final class StorageDir { * @param blockId Id of the block */ public void accessBlock(long blockId) { - if (containsBlock(blockId)) { - mLastBlockAccessTimeMs.put(blockId, System.currentTimeMillis()); - } + mLastBlockAccessTimeMs.put(blockId, System.currentTimeMillis()); } /** @@ -120,29 +113,10 @@ public void accessBlock(long blockId) { * * @param blockId Id of the block * @param sizeBytes size of the block in bytes - * @param need to be reported During heart beat with master */ - private void addBlockId(long blockId, long sizeBytes, boolean report) { - addBlockId(blockId, sizeBytes, System.currentTimeMillis(), report); - } - - /** - * Add information of a block in current StorageDir - * - * @param blockId Id of the block - * @param sizeBytes size of the block in bytes - * @param accessTime access time of the block - * @param need to be reported During heart beat with master - */ - private void addBlockId(long blockId, long sizeBytes, long accessTime, boolean report) { - mLastBlockAccessTimeMs.put(blockId, accessTime); - if (mBlockSizes.containsKey(blockId)) { - mSpaceCounter.returnUsedBytes(mBlockSizes.remove(blockId)); - } + private void addBlockId(long blockId, long sizeBytes) { + accessBlock(blockId); mBlockSizes.put(blockId, sizeBytes); - if (report) { - mAddedBlockIdList.add(blockId); - } } /** @@ -157,20 +131,9 @@ public boolean cacheBlock(long userId, long blockId) throws IOException { String srcPath = getUserTempFilePath(userId, blockId); String dstPath = getBlockFilePath(blockId); long blockSize = mFs.getFileSize(srcPath); - if (blockSize < 0) { - throw new IOException("Negative size of block! size:" + blockSize); - } boolean result = mFs.rename(srcPath, dstPath); if (result) { - addBlockId(blockId, blockSize, false); - Long used; - do { - used = mUserAvailableBytes.get(userId); - if (used == null) { - LOG.error("Failed to record allocated space for user! Id:" + userId); - break; - } - } while (!mUserAvailableBytes.replace(userId, used, used - blockSize)); + addBlockId(blockId, blockSize); } return result; } @@ -187,7 +150,7 @@ public void checkStatus(List removedUsers) { for (long blockId : blockIds) { mUserPerLockedBlock.remove(blockId, userId); } - returnSpace(userId); + mUserAllocatedBytes.remove(userId); } } @@ -198,7 +161,7 @@ public void checkStatus(List removedUsers) { * @return true if StorageDir contains the block, false otherwise */ public boolean containsBlock(long blockId) { - return mLastBlockAccessTimeMs.containsKey(blockId) || isBlockLocked(blockId); + return mLastBlockAccessTimeMs.containsKey(blockId); } /** @@ -209,7 +172,7 @@ public boolean containsBlock(long blockId) { * @return true if success, false otherwise * @throws IOException */ - public boolean copyBlock(long blockId, StorageDir dstDir) throws IOException { + boolean copyBlock(long blockId, StorageDir dstDir) throws IOException { long size = getBlockSize(blockId); if (size == -1) { LOG.error("Block file doesn't exist! blockId:" + blockId); @@ -226,8 +189,7 @@ public boolean copyBlock(long blockId, StorageDir dstDir) throws IOException { closer.close(); } if (copySuccess) { - long accessTime = mLastBlockAccessTimeMs.get(blockId); - dstDir.addBlockId(blockId, size, accessTime, true); + dstDir.addBlockId(blockId, size); } return copySuccess; } @@ -245,11 +207,8 @@ public boolean deleteBlock(long blockId) throws IOException { String blockfile = getBlockFilePath(blockId); boolean result = false; try { - // Should check lock status here if (!isBlockLocked(blockId)) { result = mFs.delete(blockfile, true); - } else { - mToRemoveBlockIdSet.add(blockId); } } finally { if (result) { @@ -262,7 +221,7 @@ public boolean deleteBlock(long blockId) throws IOException { } return result; } else { - LOG.warn("Block " + blockId + " does not exist in current StorageDir."); + LOG.error("Block " + blockId + " does not exist in current StorageDir."); return false; } } @@ -274,21 +233,8 @@ public boolean deleteBlock(long blockId) throws IOException { */ private void deleteBlockId(long blockId) { mLastBlockAccessTimeMs.remove(blockId); - mSpaceCounter.returnUsedBytes(mBlockSizes.remove(blockId)); - if (mAddedBlockIdList.contains(blockId)) { - mAddedBlockIdList.remove(blockId); - } - } - - /** - * Get Ids of newly added blocks - * - * @return list of added block Ids - */ - public List getAddedBlockIdList() { - List addedBlockIdList = new ArrayList(); - mAddedBlockIdList.drainTo(addedBlockIdList); - return addedBlockIdList; + returnSpace(mBlockSizes.remove(blockId)); + mRemovedBlockIdList.add(blockId); } /** @@ -315,7 +261,6 @@ public ByteBuffer getBlockData(long blockId, long offset, int length) throws IOE return bh.read(offset, length); } finally { bh.close(); - accessBlock(blockId); } } @@ -325,7 +270,7 @@ public ByteBuffer getBlockData(long blockId, long offset, int length) throws IOE * @param blockId Id of the block * @return file path of the block */ - public String getBlockFilePath(long blockId) { + String getBlockFilePath(long blockId) { return mDataPath.join("" + blockId).toString(); } @@ -336,7 +281,7 @@ public String getBlockFilePath(long blockId) { * @return block handler of the block file * @throws IOException */ - public BlockHandler getBlockHandler(long blockId) throws IOException { + private BlockHandler getBlockHandler(long blockId) throws IOException { String filePath = getBlockFilePath(blockId); try { return BlockHandler.get(filePath); @@ -430,6 +375,17 @@ public long getLockedSizeBytes() { return lockedBytes; } + /** + * Get Ids of removed blocks + * + * @return list of removed block Ids + */ + public List getRemovedBlockIdList() { + List removedBlockIds = new ArrayList(); + mRemovedBlockIdList.drainTo(removedBlockIds); + return removedBlockIds; + } + /** * Get Id of current StorageDir * @@ -530,9 +486,9 @@ public void initailize() throws IOException { long fileSize = mFs.getFileSize(path); LOG.debug("File " + cnt + ": " + path + " with size " + fileSize + " Bs."); long blockId = CommonUtils.getBlockIdFromFileName(name); - boolean success = mSpaceCounter.requestSpaceBytes(fileSize); + boolean success = requestSpace(fileSize); if (success) { - addBlockId(blockId, fileSize, true); + addBlockId(blockId, fileSize); } else { mFs.delete(path, true); throw new RuntimeException("Pre-existing files exceed storage capacity."); @@ -560,7 +516,7 @@ public boolean isBlockLocked(long blockId) { * @return true if success, false otherwise */ public boolean lockBlock(long blockId, long userId) { - if (!containsBlock(blockId)) { + if (!containsBlock(blockId) && !isBlockLocked(blockId)) { return false; } mUserPerLockedBlock.put(blockId, userId); @@ -585,6 +541,16 @@ public boolean moveBlock(long blockId, StorageDir dstDir) throws IOException { } } + /** + * Request space from current StorageDir + * + * @param size request size in bytes + * @return true if success, false otherwise + */ + public boolean requestSpace(long size) { + return mSpaceCounter.requestSpaceBytes(size); + } + /** * Request space from current StorageDir by some user * @@ -593,12 +559,12 @@ public boolean moveBlock(long blockId, StorageDir dstDir) throws IOException { * @return true if success, false otherwise */ public boolean requestSpace(long userId, long size) { - boolean result = mSpaceCounter.requestSpaceBytes(size); - if (result && userId != Users.EVICT_USER_ID) { - Long used = mUserAvailableBytes.putIfAbsent(userId, size); + boolean result = requestSpace(size); + if (result) { + Long used = mUserAllocatedBytes.putIfAbsent(userId, size); if (used != null) { - while (!mUserAvailableBytes.replace(userId, used, used + size)) { - used = mUserAvailableBytes.get(userId); + while (!mUserAllocatedBytes.replace(userId, used, used + size)) { + used = mUserAllocatedBytes.get(userId); if (used == null) { LOG.error("Failed to request space! unknown user Id:" + userId); break; @@ -612,16 +578,10 @@ public boolean requestSpace(long userId, long size) { /** * Return space to current StorageDir * - * @param userId Id of the user to return space + * @param size size to return in bytes */ - private void returnSpace(long userId) { - Long returnedSize = mUserAvailableBytes.remove(userId); - if (returnedSize != null) { - LOG.info(String.format("Return space(%d) from removed user(%d):", returnedSize, userId)); - mSpaceCounter.returnUsedBytes(returnedSize); - } else { - LOG.warn("No space allocation record for removed user! Id:" + userId); - } + public void returnSpace(long size) { + mSpaceCounter.returnUsedBytes(size); } /** @@ -631,15 +591,15 @@ private void returnSpace(long userId) { * @param size size to return in bytes */ public void returnSpace(long userId, long size) { - mSpaceCounter.returnUsedBytes(size); + returnSpace(size); Long used; do { - used = mUserAvailableBytes.get(userId); + used = mUserAllocatedBytes.get(userId); if (used == null) { LOG.error("Failed to return space! unknown user Id:" + userId); break; } - } while (!mUserAvailableBytes.replace(userId, used, used - size)); + } while (!mUserAllocatedBytes.replace(userId, used, used - size)); } /** @@ -650,20 +610,11 @@ public void returnSpace(long userId, long size) { * @return true if success, false otherwise */ public boolean unlockBlock(long blockId, long userId) { - if (!containsBlock(blockId)) { + if (!containsBlock(blockId) && !isBlockLocked(blockId)) { return false; } mUserPerLockedBlock.remove(blockId, userId); mLockedBlocksPerUser.remove(userId, blockId); - if (!mUserPerLockedBlock.containsKey(blockId) && mToRemoveBlockIdSet.contains(blockId)) { - try { - if (deleteBlock(blockId)) { - mToRemoveBlockIdSet.remove(blockId); - } - } catch (IOException e) { - LOG.error(e.getMessage()); - } - } return true; } } diff --git a/core/src/main/java/tachyon/worker/hierarchy/StorageTier.java b/core/src/main/java/tachyon/worker/hierarchy/StorageTier.java index 59bca476a32a..a71af20a9af1 100644 --- a/core/src/main/java/tachyon/worker/hierarchy/StorageTier.java +++ b/core/src/main/java/tachyon/worker/hierarchy/StorageTier.java @@ -26,7 +26,6 @@ import tachyon.Pair; import tachyon.StorageDirId; import tachyon.StorageLevelAlias; -import tachyon.Users; import tachyon.conf.UserConf; import tachyon.conf.WorkerConf; import tachyon.worker.eviction.EvictStrategies; @@ -219,13 +218,12 @@ public boolean isLastTier() { * @param userId id of the user * @param requestSizeBytes size to request in bytes * @param pinList list of pinned files - * @param removedBlockIds list of blocks which are removed from Tachyon * @return the StorageDir assigned. * @throws IOException */ - public StorageDir requestSpace(long userId, long requestSizeBytes, Set pinList, - List removedBlockIds) throws IOException { - return requestSpace(mStorageDirs, userId, requestSizeBytes, pinList, removedBlockIds); + public StorageDir requestSpace(long userId, long requestSizeBytes, Set pinList) + throws IOException { + return requestSpace(mStorageDirs, userId, requestSizeBytes, pinList); } /** @@ -235,19 +233,17 @@ public StorageDir requestSpace(long userId, long requestSizeBytes, Set * @param userId id of the user * @param requestSizeBytes size to request in bytes * @param pinList list of pinned files - * @param removedBlockIds list of blocks which are removed from Tachyon * @return true if allocate successfully, false otherwise. * @throws IOException */ public boolean requestSpace(StorageDir storageDir, long userId, long requestSizeBytes, - Set pinList, List removedBlockIds) throws IOException { + Set pinList) throws IOException { if (StorageDirId.getStorageLevel(storageDir.getStorageDirId()) != mStorageLevel) { return false; } StorageDir[] dirCandidates = new StorageDir[1]; dirCandidates[0] = storageDir; - return storageDir == requestSpace(dirCandidates, userId, requestSizeBytes, pinList, - removedBlockIds); + return storageDir == requestSpace(dirCandidates, userId, requestSizeBytes, pinList); } /** @@ -257,14 +253,12 @@ public boolean requestSpace(StorageDir storageDir, long userId, long requestSize * @param userId id of the user * @param requestSizeBytes size to request in bytes * @param pinList list of pinned files - * @param removedBlockIds list of blocks which are removed from Tachyon * @return the StorageDir assigned. * @throws IOException */ - // TODO make block eviction asynchronous, then no need to be synchronized - private synchronized StorageDir requestSpace(StorageDir[] dirCandidates, long userId, - long requestSizeBytes, Set pinList, List removedBlockIds) - throws IOException { + // TODO make block eviction asynchronous + private StorageDir requestSpace(StorageDir[] dirCandidates, long userId, long requestSizeBytes, + Set pinList) throws IOException { StorageDir dirSelected = mSpaceAllocator.getStorageDir(dirCandidates, userId, requestSizeBytes); if (dirSelected != null) { return dirSelected; @@ -283,14 +277,13 @@ private synchronized StorageDir requestSpace(StorageDir[] dirCandidates, long us long blockId = blockInfo.getBlockId(); if (isLastTier()) { srcDir.deleteBlock(blockId); - removedBlockIds.add(blockId); + } else if (mNextStorageTier.containsBlock(blockId)) { + srcDir.deleteBlock(blockId); } else { StorageDir dstDir = - mNextStorageTier.requestSpace(Users.EVICT_USER_ID, blockInfo.getBlockSize(), - pinList, removedBlockIds); + mNextStorageTier.requestSpace(userId, blockInfo.getBlockSize(), pinList); srcDir.moveBlock(blockId, dstDir); } - LOG.debug("Evicted block Id:" + blockId); } } if (dirSelected.requestSpace(userId, requestSizeBytes)) { @@ -301,9 +294,7 @@ private synchronized StorageDir requestSpace(StorageDir[] dirCandidates, long us } } } - LOG.warn("No StorageDir is allocated! requestSize:" + requestSizeBytes + " usedSpace:" - + getUsedBytes() + " capacity:" + getCapacityBytes()); - return null; + throw new IOException("No StorageDir is allocated!"); } @Override diff --git a/core/src/main/java/tachyon/worker/netty/BlockResponse.java b/core/src/main/java/tachyon/worker/netty/BlockResponse.java index e2a86d10639c..5b63821ff57b 100644 --- a/core/src/main/java/tachyon/worker/netty/BlockResponse.java +++ b/core/src/main/java/tachyon/worker/netty/BlockResponse.java @@ -15,7 +15,7 @@ package tachyon.worker.netty; -import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; import java.util.List; @@ -29,7 +29,6 @@ import com.google.common.primitives.Shorts; import tachyon.conf.WorkerConf; -import tachyon.worker.BlockHandler; import tachyon.worker.nio.DataServerMessage; /** @@ -57,22 +56,17 @@ private ByteBuf createHeader(final ChannelHandlerContext ctx, final BlockRespons protected void encode(final ChannelHandlerContext ctx, final BlockResponse msg, final List out) throws Exception { out.add(createHeader(ctx, msg)); - BlockHandler handler = msg.getHandler(); - if (handler != null) { + if (msg.getChannel() != null) { switch (WorkerConf.get().NETTY_FILE_TRANSFER_TYPE) { case MAPPED: - ByteBuffer data = handler.read(msg.getOffset(), (int) msg.getLength()); + MappedByteBuffer data = + msg.getChannel().map(FileChannel.MapMode.READ_ONLY, msg.getOffset(), + msg.getLength()); out.add(Unpooled.wrappedBuffer(data)); - handler.close(); + msg.getChannel().close(); break; case TRANSFER: - if (handler.getChannel() instanceof FileChannel) { - out.add(new DefaultFileRegion((FileChannel) handler.getChannel(), msg.getOffset(), - msg.getLength())); - } else { - handler.close(); - throw new Exception("Only FileChannel is supported!"); - } + out.add(new DefaultFileRegion(msg.getChannel(), msg.getOffset(), msg.getLength())); break; default: throw new AssertionError("Unknown file transfer type: " @@ -86,7 +80,7 @@ protected void encode(final ChannelHandlerContext ctx, final BlockResponse msg, * Creates a {@link tachyon.worker.netty.BlockResponse} that represents a error case for the given * block. */ - public static BlockResponse createErrorResponse(final long storageDirId, final long blockId) { + public static BlockResponse createErrorResponse(final long blockId) { return new BlockResponse(-blockId, 0, 0, null); } @@ -95,22 +89,21 @@ public static BlockResponse createErrorResponse(final long storageDirId, final l private final long mLength; - private final BlockHandler mHandler; + private final FileChannel mChannel; - public BlockResponse(long blockId, long offset, long length, - BlockHandler handler) { + public BlockResponse(long blockId, long offset, long length, FileChannel channel) { mBlockId = blockId; mOffset = offset; mLength = length; - mHandler = handler; + mChannel = channel; } public long getBlockId() { return mBlockId; } - public BlockHandler getHandler() { - return mHandler; + public FileChannel getChannel() { + return mChannel; } public long getLength() { diff --git a/core/src/main/java/tachyon/worker/netty/DataServerHandler.java b/core/src/main/java/tachyon/worker/netty/DataServerHandler.java index a0273ae38f7b..9781b89f3444 100644 --- a/core/src/main/java/tachyon/worker/netty/DataServerHandler.java +++ b/core/src/main/java/tachyon/worker/netty/DataServerHandler.java @@ -16,6 +16,9 @@ package tachyon.worker.netty; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; + import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandler; @@ -25,11 +28,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.io.Closeables; + import tachyon.Constants; -import tachyon.worker.BlockHandler; +import tachyon.conf.WorkerConf; +import tachyon.util.CommonUtils; import tachyon.worker.BlocksLocker; -import tachyon.worker.WorkerStorage; -import tachyon.worker.hierarchy.StorageDir; /** * Main logic for the read path. This class consumes {@link tachyon.worker.netty.BlockRequest} @@ -40,11 +44,9 @@ public final class DataServerHandler extends ChannelInboundHandlerAdapter { private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); private final BlocksLocker mLocker; - private final WorkerStorage mWorkerStorage; - public DataServerHandler(BlocksLocker locker, WorkerStorage workerStorage) { + public DataServerHandler(BlocksLocker locker) { mLocker = locker; - mWorkerStorage = workerStorage; } @Override @@ -55,33 +57,36 @@ public void channelRead(final ChannelHandlerContext ctx, final Object msg) throw final long blockId = req.getBlockId(); final long offset = req.getOffset(); final long len = req.getLength(); - final int lockId = mLocker.getLockId(); - final long storageDirIdLocked = mLocker.lock(blockId, lockId); - BlockHandler handler = null; + final int lockId = mLocker.lock(blockId); + + RandomAccessFile file = null; try { validateInput(req); - StorageDir storageDir = mWorkerStorage.getStorageDirById(storageDirIdLocked); - handler = storageDir.getBlockHandler(blockId); - final long fileLength = handler.getLength(); + String filePath = CommonUtils.concat(WorkerConf.get().DATA_FOLDER, blockId); + LOG.info("Try to response remote request by reading from " + filePath); + + file = new RandomAccessFile(filePath, "r"); + long fileLength = file.length(); validateBounds(req, fileLength); + final long readLength = returnLength(offset, len, fileLength); + + FileChannel channel = file.getChannel(); ChannelFuture future = - ctx.writeAndFlush(new BlockResponse(blockId, offset, readLength, handler)); + ctx.writeAndFlush(new BlockResponse(blockId, offset, readLength, channel)); future.addListener(ChannelFutureListener.CLOSE); - future.addListener(new ClosableResourceChannelListener(handler)); - storageDir.accessBlock(blockId); - LOG.info("Response remote request by reading from " + storageDir.getBlockFilePath(blockId) - + " preparation done."); + future.addListener(new ClosableResourceChannelListener(file)); + LOG.info("Response remote request by reading from " + filePath + " preparation done."); } catch (Exception e) { // TODO This is a trick for now. The data may have been removed before remote retrieving. LOG.error("The file is not here : " + e.getMessage(), e); - BlockResponse resp = BlockResponse.createErrorResponse(storageDirIdLocked, blockId); + BlockResponse resp = BlockResponse.createErrorResponse(blockId); ChannelFuture future = ctx.writeAndFlush(resp); future.addListener(ChannelFutureListener.CLOSE); - if (handler != null) { - handler.close(); + if (file != null) { + Closeables.close(file, true); } } finally { mLocker.unlock(blockId, lockId); diff --git a/core/src/main/java/tachyon/worker/netty/NettyDataServer.java b/core/src/main/java/tachyon/worker/netty/NettyDataServer.java index c10fcdecf6da..f51c0745a69b 100644 --- a/core/src/main/java/tachyon/worker/netty/NettyDataServer.java +++ b/core/src/main/java/tachyon/worker/netty/NettyDataServer.java @@ -36,7 +36,6 @@ import tachyon.util.ThreadFactoryUtils; import tachyon.worker.BlocksLocker; import tachyon.worker.DataServer; -import tachyon.worker.WorkerStorage; /** * Runs a netty server that will response to block requests. @@ -46,9 +45,8 @@ public final class NettyDataServer implements DataServer { private final ChannelFuture mChannelFuture; - public NettyDataServer(final SocketAddress address, final BlocksLocker locker, - WorkerStorage workerStorage) { - mBootstrap = createBootstrap().childHandler(new PipelineHandler(locker, workerStorage)); + public NettyDataServer(final SocketAddress address, final BlocksLocker locker) { + mBootstrap = createBootstrap().childHandler(new PipelineHandler(locker)); try { mChannelFuture = mBootstrap.bind(address).sync(); diff --git a/core/src/main/java/tachyon/worker/netty/PipelineHandler.java b/core/src/main/java/tachyon/worker/netty/PipelineHandler.java index ea0f815632c9..2d3d11b75137 100644 --- a/core/src/main/java/tachyon/worker/netty/PipelineHandler.java +++ b/core/src/main/java/tachyon/worker/netty/PipelineHandler.java @@ -21,18 +21,15 @@ import io.netty.handler.stream.ChunkedWriteHandler; import tachyon.worker.BlocksLocker; -import tachyon.worker.WorkerStorage; /** * Adds the block server's pipeline into the channel. */ public final class PipelineHandler extends ChannelInitializer { private final BlocksLocker mLocker; - private final WorkerStorage mWorkerStorage; - public PipelineHandler(BlocksLocker locker, WorkerStorage workerStorage) { + public PipelineHandler(BlocksLocker locker) { mLocker = locker; - mWorkerStorage = workerStorage; } @Override @@ -41,6 +38,6 @@ protected void initChannel(SocketChannel ch) throws Exception { pipeline.addLast("nioChunkedWriter", new ChunkedWriteHandler()); pipeline.addLast("blockRequestDecoder", new BlockRequest.Decoder()); pipeline.addLast("blockResponseEncoder", new BlockResponse.Encoder()); - pipeline.addLast("dataServerHandler", new DataServerHandler(mLocker, mWorkerStorage)); + pipeline.addLast("dataServerHandler", new DataServerHandler(mLocker)); } } diff --git a/core/src/main/java/tachyon/worker/nio/DataServerMessage.java b/core/src/main/java/tachyon/worker/nio/DataServerMessage.java index 755787d79aaa..e602bd9dd510 100644 --- a/core/src/main/java/tachyon/worker/nio/DataServerMessage.java +++ b/core/src/main/java/tachyon/worker/nio/DataServerMessage.java @@ -16,7 +16,9 @@ package tachyon.worker.nio; import java.io.IOException; +import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.nio.channels.SocketChannel; import org.slf4j.Logger; @@ -26,6 +28,8 @@ import tachyon.Constants; import tachyon.client.TachyonByteBuffer; +import tachyon.conf.WorkerConf; +import tachyon.util.CommonUtils; /** * The message type used to send data request and response for remote data. @@ -35,7 +39,7 @@ public class DataServerMessage { public static final short DATA_SERVER_RESPONSE_MESSAGE = 2; private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); - + private static final int HEADER_LENGTH = 26; /** @@ -72,8 +76,7 @@ public static DataServerMessage createBlockRequestMessage(long blockId) { * to the block's end. * @return The created block request message */ - public static DataServerMessage createBlockRequestMessage(long blockId, - long offset, long len) { + public static DataServerMessage createBlockRequestMessage(long blockId, long offset, long len) { DataServerMessage ret = new DataServerMessage(true, DATA_SERVER_REQUEST_MESSAGE); ret.mHeader = ByteBuffer.allocate(HEADER_LENGTH); @@ -95,9 +98,8 @@ public static DataServerMessage createBlockRequestMessage(long blockId, * @param blockId The id of the block * @return The created block response message */ - public static DataServerMessage createBlockResponseMessage(boolean toSend, long blockId, - ByteBuffer data) { - return createBlockResponseMessage(toSend, blockId, 0, -1, data); + public static DataServerMessage createBlockResponseMessage(boolean toSend, long blockId) { + return createBlockResponseMessage(toSend, blockId, 0, -1); } /** @@ -114,28 +116,63 @@ public static DataServerMessage createBlockResponseMessage(boolean toSend, long * @return The created block response message */ public static DataServerMessage createBlockResponseMessage(boolean toSend, long blockId, - long offset, long len, ByteBuffer data) { + long offset, long len) { DataServerMessage ret = new DataServerMessage(toSend, DATA_SERVER_RESPONSE_MESSAGE); if (toSend) { - if (data != null) { + ret.mBlockId = blockId; + + try { + if (offset < 0) { + throw new IOException("Offset can not be negative: " + offset); + } + if (len < 0 && len != -1) { + throw new IOException("Length can not be negative except -1: " + len); + } + + String filePath = CommonUtils.concat(WorkerConf.get().DATA_FOLDER, blockId); + LOG.info("Try to response remote request by reading from " + filePath); + RandomAccessFile file = new RandomAccessFile(filePath, "r"); + + long fileLength = file.length(); + String error = null; + if (offset > fileLength) { + error = String.format("Offset(%d) is larger than file length(%d)", offset, fileLength); + } + if (error == null && len != -1 && offset + len > fileLength) { + error = + String.format("Offset(%d) plus length(%d) is larger than file length(%d)", offset, + len, fileLength); + } + if (error != null) { + file.close(); + throw new IOException(error); + } + + if (len == -1) { + len = fileLength - offset; + } + ret.mHeader = ByteBuffer.allocate(HEADER_LENGTH); - ret.mBlockId = blockId; ret.mOffset = offset; ret.mLength = len; + FileChannel channel = file.getChannel(); ret.mTachyonData = null; - ret.mData = data; + ret.mData = channel.map(FileChannel.MapMode.READ_ONLY, offset, len); + channel.close(); + file.close(); ret.mIsMessageReady = true; ret.generateHeader(); - } else { + LOG.info("Response remote request by reading from " + filePath + " preparation done."); + } catch (Exception e) { // TODO This is a trick for now. The data may have been removed before remote retrieving. - ret.mBlockId = -blockId; + ret.mBlockId = -ret.mBlockId; ret.mLength = 0; ret.mHeader = ByteBuffer.allocate(HEADER_LENGTH); ret.mData = ByteBuffer.allocate(0); ret.mIsMessageReady = true; ret.generateHeader(); - LOG.error("The file is not here"); + LOG.error("The file is not here : " + e.getMessage(), e); } } else { ret.mHeader = ByteBuffer.allocate(HEADER_LENGTH); @@ -150,7 +187,6 @@ public static DataServerMessage createBlockResponseMessage(boolean toSend, long private boolean mIsMessageReady; private ByteBuffer mHeader; - private long mBlockId; private long mOffset; diff --git a/core/src/main/java/tachyon/worker/nio/NIODataServer.java b/core/src/main/java/tachyon/worker/nio/NIODataServer.java index c6ebc0e01c8f..9fcf2e621341 100644 --- a/core/src/main/java/tachyon/worker/nio/NIODataServer.java +++ b/core/src/main/java/tachyon/worker/nio/NIODataServer.java @@ -17,7 +17,6 @@ import java.io.IOException; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.nio.channels.SelectionKey; import java.nio.channels.Selector; import java.nio.channels.ServerSocketChannel; @@ -37,8 +36,6 @@ import tachyon.conf.CommonConf; import tachyon.worker.BlocksLocker; import tachyon.worker.DataServer; -import tachyon.worker.WorkerStorage; -import tachyon.worker.hierarchy.StorageDir; /** * The Server to serve data file read request from remote machines. The current implementation is @@ -48,7 +45,7 @@ public class NIODataServer implements Runnable, DataServer { private static final Logger LOG = LoggerFactory.getLogger(Constants.LOGGER_TYPE); // The host:port combination to listen on - private final InetSocketAddress mAddress; + private InetSocketAddress mAddress; // The channel on which we will accept connections private ServerSocketChannel mServerChannel; @@ -56,9 +53,9 @@ public class NIODataServer implements Runnable, DataServer { // The selector we will be monitoring. private Selector mSelector; - private final Map mSendingData = Collections + private Map mSendingData = Collections .synchronizedMap(new HashMap()); - private final Map mReceivingData = Collections + private Map mReceivingData = Collections .synchronizedMap(new HashMap()); // The blocks locker manager. @@ -67,22 +64,18 @@ public class NIODataServer implements Runnable, DataServer { private volatile boolean mShutdown = false; private volatile boolean mShutdowned = false; - private final WorkerStorage mWorkerStorage; /** * Create a data server with direct access to worker storage. * * @param address The address of the data server. * @param locker The lock system for lock blocks. - * @param workerStorage The WorkerStorage of current worker */ - public NIODataServer(InetSocketAddress address, BlocksLocker locker, - WorkerStorage workerStorage) { + public NIODataServer(InetSocketAddress address, BlocksLocker locker) { LOG.info("Starting DataServer @ " + address); CommonConf.assertValidPort(address); mAddress = address; mBlockLocker = locker; - mWorkerStorage = workerStorage; try { mSelector = initSelector(); mListenerThread = new Thread(this); @@ -215,28 +208,11 @@ private void read(SelectionKey key) throws IOException { key.interestOps(SelectionKey.OP_WRITE); LOG.info("Get request for " + tMessage.getBlockId()); - final long blockId = tMessage.getBlockId(); - final int lockId = mBlockLocker.getLockId(); - final long storageDirIdLocked = mBlockLocker.lock(blockId, lockId); - - StorageDir storageDir = mWorkerStorage.getStorageDirById(storageDirIdLocked); - ByteBuffer data = null; - int dataLen = 0; - try { - data = - storageDir.getBlockData(tMessage.getBlockId(), tMessage.getOffset(), - (int) tMessage.getLength()); - storageDir.accessBlock(tMessage.getBlockId()); - dataLen = data.limit(); - } catch (Exception e) { - LOG.error(e.getMessage()); - data = null; - } + int lockId = mBlockLocker.lock(tMessage.getBlockId()); DataServerMessage tResponseMessage = - DataServerMessage.createBlockResponseMessage(true, blockId, tMessage.getOffset(), - dataLen, data); + DataServerMessage.createBlockResponseMessage(true, tMessage.getBlockId(), + tMessage.getOffset(), tMessage.getLength()); tResponseMessage.setLockId(lockId); - mBlockLocker.unlock(blockId, lockId); mSendingData.put(socketChannel, tResponseMessage); } } diff --git a/core/src/test/java/tachyon/worker/BlockHandlerLocalTest.java b/core/src/test/java/tachyon/client/BlockHandlerLocalTest.java similarity index 85% rename from core/src/test/java/tachyon/worker/BlockHandlerLocalTest.java rename to core/src/test/java/tachyon/client/BlockHandlerLocalTest.java index 93c2d24e4f30..a7dfb81218a7 100644 --- a/core/src/test/java/tachyon/worker/BlockHandlerLocalTest.java +++ b/core/src/test/java/tachyon/client/BlockHandlerLocalTest.java @@ -12,7 +12,7 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -package tachyon.worker; +package tachyon.client; import java.io.IOException; import java.nio.ByteBuffer; @@ -24,11 +24,7 @@ import tachyon.TachyonURI; import tachyon.TestUtils; -import tachyon.client.TachyonFS; -import tachyon.client.TachyonFile; -import tachyon.client.WriteType; import tachyon.master.LocalTachyonCluster; -import tachyon.thrift.ClientLocationInfo; import tachyon.util.CommonUtils; public class BlockHandlerLocalTest { @@ -57,13 +53,12 @@ public void directByteBufferWriteTest() throws IOException { int fileId = mTfs.createFile(new TachyonURI("/root/testFile")); long blockId = mTfs.getBlockId(fileId, 0); - ClientLocationInfo locationInfo = mTfs.requestSpace(100); - String tempFolder = mTfs.createAndGetUserLocalTempFolder(locationInfo.getPath()); - String filename = CommonUtils.concat(tempFolder, blockId); + String localFolder = mTfs.createAndGetUserLocalTempFolder().getPath(); + String filename = CommonUtils.concat(localFolder, blockId); BlockHandler handler = BlockHandler.get(filename); try { handler.append(0, buf); - mTfs.cacheBlock(locationInfo.getStorageDirId(), blockId); + mTfs.cacheBlock(blockId); TachyonFile file = mTfs.getFile(fileId); long fileLen = file.length(); Assert.assertEquals(100, fileLen); @@ -77,14 +72,13 @@ public void directByteBufferWriteTest() throws IOException { public void heapByteBufferwriteTest() throws IOException { int fileId = mTfs.createFile(new TachyonURI("/root/testFile")); long blockId = mTfs.getBlockId(fileId, 0); - ClientLocationInfo locationInfo = mTfs.requestSpace(100); - String tempFolder = mTfs.createAndGetUserLocalTempFolder(locationInfo.getPath()); - String filename = CommonUtils.concat(tempFolder, blockId); + String localFolder = mTfs.createAndGetUserLocalTempFolder().getPath(); + String filename = CommonUtils.concat(localFolder, blockId); BlockHandler handler = BlockHandler.get(filename); byte[] buf = TestUtils.getIncreasingByteArray(100); try { handler.append(0, ByteBuffer.wrap(buf)); - mTfs.cacheBlock(locationInfo.getStorageDirId(), blockId); + mTfs.cacheBlock(blockId); TachyonFile file = mTfs.getFile(fileId); long fileLen = file.length(); Assert.assertEquals(100, fileLen); diff --git a/core/src/test/java/tachyon/client/TachyonFSTest.java b/core/src/test/java/tachyon/client/TachyonFSTest.java index 6741f11ccf66..dd4585862df9 100644 --- a/core/src/test/java/tachyon/client/TachyonFSTest.java +++ b/core/src/test/java/tachyon/client/TachyonFSTest.java @@ -16,7 +16,6 @@ import java.io.IOException; import java.io.OutputStream; -import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -26,7 +25,6 @@ import org.junit.Before; import org.junit.Test; -import tachyon.Constants; import tachyon.TachyonURI; import tachyon.TestUtils; import tachyon.UnderFileSystem; @@ -457,10 +455,9 @@ public void renameFileTest3() throws IOException { } @Test - public void toStringTest() throws IOException, InterruptedException { - InetSocketAddress masterAddress = mLocalTachyonCluster.getMasterInfo().getMasterAddress(); - TachyonFS tfs = TachyonFS.get(new TachyonURI(mLocalTachyonCluster.getMasterUri())); - Assert.assertEquals(tfs.toString(), Constants.HEADER + masterAddress.toString()); + public void toStringTest() throws IOException { + TachyonFS tfs = TachyonFS.get(new TachyonURI("tachyon://127.0.0.1:19998")); + Assert.assertEquals(tfs.toString(), "tachyon:///127.0.0.1:19998"); } @Test diff --git a/core/src/test/java/tachyon/master/BlockInfoTest.java b/core/src/test/java/tachyon/master/BlockInfoTest.java index 80fb8ac9bd74..72e72f7aa812 100644 --- a/core/src/test/java/tachyon/master/BlockInfoTest.java +++ b/core/src/test/java/tachyon/master/BlockInfoTest.java @@ -18,8 +18,6 @@ import org.junit.Test; import tachyon.Constants; -import tachyon.StorageDirId; -import tachyon.StorageLevelAlias; import tachyon.thrift.ClientBlockInfo; import tachyon.thrift.NetAddress; @@ -79,10 +77,9 @@ public void generateClientBlockInfoTest() { BlockInfo tInfo = new BlockInfo(new InodeFile("t", 100, 0, Constants.DEFAULT_BLOCK_SIZE_BYTE, System.currentTimeMillis()), 300, 800); - long storageDirId = StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0); - tInfo.addLocation(15, new NetAddress("abc", 1, 11), storageDirId); - tInfo.addLocation(22, new NetAddress("def", 2, 21), storageDirId); - tInfo.addLocation(29, new NetAddress("gh", 3, 31), storageDirId); + tInfo.addLocation(15, new NetAddress("abc", 1, 11)); + tInfo.addLocation(22, new NetAddress("def", 2, 21)); + tInfo.addLocation(29, new NetAddress("gh", 3, 31)); ClientBlockInfo clientBlockInfo = tInfo.generateClientBlockInfo(); Assert.assertEquals((long) Constants.DEFAULT_BLOCK_SIZE_BYTE * 300, clientBlockInfo.offset); Assert.assertEquals(800, clientBlockInfo.length); @@ -94,18 +91,17 @@ public void localtionTest() { BlockInfo tInfo = new BlockInfo(new InodeFile("t", 100, 0, Constants.DEFAULT_BLOCK_SIZE_BYTE, System.currentTimeMillis()), 300, 800); - long storageDirId = StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0); - tInfo.addLocation(15, new NetAddress("abc", 1, 11), storageDirId); + tInfo.addLocation(15, new NetAddress("abc", 1, 11)); Assert.assertEquals(1, tInfo.getLocations().size()); - tInfo.addLocation(22, new NetAddress("def", 2, 21), storageDirId); + tInfo.addLocation(22, new NetAddress("def", 2, 21)); Assert.assertEquals(2, tInfo.getLocations().size()); - tInfo.addLocation(29, new NetAddress("gh", 3, 31), storageDirId); + tInfo.addLocation(29, new NetAddress("gh", 3, 31)); Assert.assertEquals(3, tInfo.getLocations().size()); - tInfo.addLocation(15, new NetAddress("abc", 1, 11), storageDirId); + tInfo.addLocation(15, new NetAddress("abc", 1, 11)); Assert.assertEquals(3, tInfo.getLocations().size()); - tInfo.addLocation(22, new NetAddress("def", 2, 21), storageDirId); + tInfo.addLocation(22, new NetAddress("def", 2, 21)); Assert.assertEquals(3, tInfo.getLocations().size()); - tInfo.addLocation(29, new NetAddress("gh", 3, 31), storageDirId); + tInfo.addLocation(29, new NetAddress("gh", 3, 31)); Assert.assertEquals(3, tInfo.getLocations().size()); tInfo.removeLocation(15); Assert.assertEquals(2, tInfo.getLocations().size()); diff --git a/core/src/test/java/tachyon/master/InodeFileTest.java b/core/src/test/java/tachyon/master/InodeFileTest.java index 793fd06e4b40..b428ecf1649e 100644 --- a/core/src/test/java/tachyon/master/InodeFileTest.java +++ b/core/src/test/java/tachyon/master/InodeFileTest.java @@ -25,8 +25,6 @@ import org.junit.Assert; import org.junit.Test; -import tachyon.StorageDirId; -import tachyon.StorageLevelAlias; import tachyon.thrift.BlockInfoException; import tachyon.thrift.NetAddress; import tachyon.thrift.SuspectedFileSizeException; @@ -63,16 +61,15 @@ public void getIdTest() { public void inMemoryLocationsTest() throws IOException, BlockInfoException { InodeFile inodeFile = new InodeFile("testFile1", 1, 0, 1000, System.currentTimeMillis()); List testAddresses = new ArrayList(3); - long storageDirId = StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0); testAddresses.add(new NetAddress("testhost1", 1000, 1001)); testAddresses.add(new NetAddress("testhost2", 2000, 2001)); testAddresses.add(new NetAddress("testhost3", 3000, 3001)); inodeFile.addBlock(new BlockInfo(inodeFile, 0, 5)); - inodeFile.addLocation(0, 1, testAddresses.get(0), storageDirId); + inodeFile.addLocation(0, 1, testAddresses.get(0)); Assert.assertEquals(1, inodeFile.getBlockLocations(0).size()); - inodeFile.addLocation(0, 2, testAddresses.get(1), storageDirId); + inodeFile.addLocation(0, 2, testAddresses.get(1)); Assert.assertEquals(2, inodeFile.getBlockLocations(0).size()); - inodeFile.addLocation(0, 3, testAddresses.get(2), storageDirId); + inodeFile.addLocation(0, 3, testAddresses.get(2)); Assert.assertEquals(3, inodeFile.getBlockLocations(0).size()); Assert.assertEquals(testAddresses, inodeFile.getBlockLocations(0)); } @@ -80,27 +77,25 @@ public void inMemoryLocationsTest() throws IOException, BlockInfoException { @Test(expected = BlockInfoException.class) public void inMemoryLocationsTestWithBlockInfoException() throws IOException, BlockInfoException { InodeFile inodeFile = new InodeFile("testFile1", 1, 0, 1000, System.currentTimeMillis()); - long storageDirId = StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0); - inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001), storageDirId); + inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001)); } @Test public void inMemoryTest() throws BlockInfoException { InodeFile inodeFile = new InodeFile("testFile1", 1, 0, 1000, System.currentTimeMillis()); inodeFile.addBlock(new BlockInfo(inodeFile, 0, 5)); - long storageDirId = StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0); Assert.assertFalse(inodeFile.isFullyInMemory()); - inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001), storageDirId); + inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001)); Assert.assertTrue(inodeFile.isFullyInMemory()); inodeFile.removeLocation(0, 1); Assert.assertFalse(inodeFile.isFullyInMemory()); - inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001), storageDirId); - inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001), storageDirId); + inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001)); + inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001)); Assert.assertTrue(inodeFile.isFullyInMemory()); inodeFile.removeLocation(0, 1); Assert.assertFalse(inodeFile.isFullyInMemory()); - inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001), storageDirId); - inodeFile.addLocation(0, 2, new NetAddress("testhost1", 1001, 1002), storageDirId); + inodeFile.addLocation(0, 1, new NetAddress("testhost1", 1000, 1001)); + inodeFile.addLocation(0, 2, new NetAddress("testhost1", 1000, 1001)); Assert.assertTrue(inodeFile.isFullyInMemory()); inodeFile.removeLocation(0, 1); Assert.assertTrue(inodeFile.isFullyInMemory()); diff --git a/core/src/test/java/tachyon/master/LocalTachyonCluster.java b/core/src/test/java/tachyon/master/LocalTachyonCluster.java index 2d28363965e6..35590790db81 100644 --- a/core/src/test/java/tachyon/master/LocalTachyonCluster.java +++ b/core/src/test/java/tachyon/master/LocalTachyonCluster.java @@ -50,7 +50,7 @@ public static void main(String[] args) throws Exception { private TachyonWorker mWorker = null; - private final long mWorkerMemCapacityBytes; + private long mWorkerCapacityBytes; private String mTachyonHome; private String mWorkerDataFolder; @@ -60,8 +60,8 @@ public static void main(String[] args) throws Exception { private LocalTachyonMaster mMaster; - public LocalTachyonCluster(long workerMemCapacityBytes) { - mWorkerMemCapacityBytes = workerMemCapacityBytes; + public LocalTachyonCluster(long workerCapacityBytes) { + mWorkerCapacityBytes = workerCapacityBytes; } public TachyonFS getClient() throws IOException { @@ -144,10 +144,9 @@ private void mkdir(String path) throws IOException { } public void start() throws IOException { - int maxLevel = 1; mTachyonHome = File.createTempFile("Tachyon", "").getAbsoluteFile() + "U" + System.currentTimeMillis(); - mWorkerDataFolder = "/datastore"; + mWorkerDataFolder = mTachyonHome + "/ramdisk"; deleteDir(mTachyonHome); mkdir(mTachyonHome); @@ -161,30 +160,7 @@ public void start() throws IOException { System.setProperty("tachyon.worker.port", 0 + ""); System.setProperty("tachyon.worker.data.port", 0 + ""); System.setProperty("tachyon.worker.data.folder", mWorkerDataFolder); - if (System.getProperty("tachyon.worker.hierarchystore.level.max") == null) { - System.setProperty("tachyon.worker.hierarchystore.level.max", 1 + ""); - } else { - maxLevel = Integer.valueOf(System.getProperty("tachyon.worker.hierarchystore.level.max")); - } - System.setProperty("tachyon.worker.hierarchystore.level0.alias", "MEM"); - System.setProperty("tachyon.worker.hierarchystore.level0.dirs.path", mTachyonHome + "/ramdisk"); - System.setProperty("tachyon.worker.hierarchystore.level0.dirs.quota", mWorkerMemCapacityBytes - + ""); - for (int level = 1; level < maxLevel; level ++) { - String path = - System.getProperty("tachyon.worker.hierarchystore.level" + level + ".dirs.path"); - if (path == null) { - throw new IOException("Paths for StorageDirs are not set! Level:" + level); - } - String[] dirPaths = path.split(","); - String newPath = ""; - for (int i = 0; i < dirPaths.length; i ++) { - newPath += mTachyonHome + dirPaths[i] + ","; - } - System.setProperty("tachyon.worker.hierarchystore.level" + level + ".dirs.path", - newPath.substring(0, newPath.length() - 1)); - } - + System.setProperty("tachyon.worker.memory.size", mWorkerCapacityBytes + ""); System.setProperty("tachyon.worker.to.master.heartbeat.interval.ms", 15 + ""); System.setProperty("tachyon.user.remote.read.buffer.size.byte", 64 + ""); // Lower the number of threads that the cluster will spin off. @@ -214,7 +190,8 @@ public void start() throws IOException { mWorker = TachyonWorker.createWorker(new InetSocketAddress(mLocalhostName, getMasterPort()), - new InetSocketAddress(mLocalhostName, 0), 0, 1, 1, 1); + new InetSocketAddress(mLocalhostName, 0), 0, 1, 1, 1, mWorkerDataFolder, + mWorkerCapacityBytes); Runnable runWorker = new Runnable() { @Override public void run() { @@ -265,7 +242,6 @@ public void stopTFS() throws Exception { System.clearProperty("tachyon.master.server.threads"); System.clearProperty("tachyon.worker.selector.threads"); System.clearProperty("tachyon.worker.server.threads"); - System.clearProperty("tachyon.worker.hierarchystore.level.max"); System.clearProperty("tachyon.worker.network.netty.worker.threads"); System.clearProperty("tachyon.master.web.threads"); } diff --git a/core/src/test/java/tachyon/master/LocalTachyonClusterMultiMaster.java b/core/src/test/java/tachyon/master/LocalTachyonClusterMultiMaster.java index ac4f668ee14f..ef6f7e1a475a 100644 --- a/core/src/test/java/tachyon/master/LocalTachyonClusterMultiMaster.java +++ b/core/src/test/java/tachyon/master/LocalTachyonClusterMultiMaster.java @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { private TestingServer mCuratorServer = null; private int mNumOfMasters = 0; private TachyonWorker mWorker = null; - private final long mWorkerCapacityBytes; + private long mWorkerCapacityBytes; private String mTachyonHome; private String mWorkerDataFolder; @@ -134,11 +134,9 @@ private void mkdir(String path) throws IOException { } public void start() throws IOException { - int maxLevel = 1; mTachyonHome = File.createTempFile("Tachyon", "").getAbsoluteFile() + "U" + System.currentTimeMillis(); - mWorkerDataFolder = "/datastore"; - + mWorkerDataFolder = mTachyonHome + "/ramdisk"; String masterDataFolder = mTachyonHome + "/data"; String masterLogFolder = mTachyonHome + "/logs"; @@ -156,28 +154,7 @@ public void start() throws IOException { System.setProperty("tachyon.zookeeper.election.path", "/election"); System.setProperty("tachyon.zookeeper.leader.path", "/leader"); System.setProperty("tachyon.worker.data.folder", mWorkerDataFolder); - if (System.getProperty("tachyon.worker.hierarchystore.level.max") == null) { - System.setProperty("tachyon.worker.hierarchystore.level.max", 1 + ""); - } - System.setProperty("tachyon.worker.hierarchystore.level0.alias", "MEM"); - - System.setProperty("tachyon.worker.hierarchystore.level0.dirs.path", mTachyonHome + "/ramdisk"); - System - .setProperty("tachyon.worker.hierarchystore.level0.dirs.quota", mWorkerCapacityBytes + ""); - for (int level = 1; level < maxLevel; level ++) { - String path = - System.getProperty("tachyon.worker.hierarchystore.level" + level + ".dirs.path"); - if (path == null) { - throw new IOException("Paths for StorageDirs are not set! Level:" + level); - } - String[] dirPaths = path.split(","); - String newPath = ""; - for (int i = 0; i < dirPaths.length; i ++) { - newPath += mTachyonHome + dirPaths[i] + ","; - } - System.setProperty("tachyon.worker.hierarchystore.level" + level + ".dirs.path", - newPath.substring(0, newPath.length() - 1)); - } + System.setProperty("tachyon.worker.memory.size", mWorkerCapacityBytes + ""); System.setProperty("tachyon.worker.to.master.heartbeat.interval.ms", 15 + ""); CommonConf.clear(); @@ -199,7 +176,8 @@ public void start() throws IOException { mWorker = TachyonWorker.createWorker( CommonUtils.parseInetSocketAddress(mCuratorServer.getConnectString()), - new InetSocketAddress(mLocalhostName, 0), 0, 1, 1, 1); + new InetSocketAddress(mLocalhostName, 0), 0, 1, 1, 1, mWorkerDataFolder, + mWorkerCapacityBytes); Runnable runWorker = new Runnable() { @Override public void run() { diff --git a/core/src/test/java/tachyon/worker/DataServerTest.java b/core/src/test/java/tachyon/worker/DataServerTest.java index a9708fc398cf..256e31ed38aa 100644 --- a/core/src/test/java/tachyon/worker/DataServerTest.java +++ b/core/src/test/java/tachyon/worker/DataServerTest.java @@ -43,7 +43,7 @@ import tachyon.worker.nio.DataServerMessage; /** - * Unit tests for tachyon.worker.DataServer. + * Unit tests for tachyon.DataServer. */ @RunWith(Parameterized.class) public class DataServerTest { @@ -222,8 +222,7 @@ private DataServerMessage request(final ClientBlockInfo block, final long offset sendMsg.send(socketChannel); } DataServerMessage recvMsg = - DataServerMessage.createBlockResponseMessage(false, block.blockId, offset, - length, null); + DataServerMessage.createBlockResponseMessage(false, block.blockId, offset, length); while (!recvMsg.isMessageReady()) { int numRead = recvMsg.recv(socketChannel); if (numRead == -1) { diff --git a/core/src/test/java/tachyon/worker/WorkerServiceHandlerTest.java b/core/src/test/java/tachyon/worker/WorkerServiceHandlerTest.java index 1b13f7ca6fe6..9bad8ac25e34 100644 --- a/core/src/test/java/tachyon/worker/WorkerServiceHandlerTest.java +++ b/core/src/test/java/tachyon/worker/WorkerServiceHandlerTest.java @@ -22,8 +22,6 @@ import org.junit.Before; import org.junit.Test; -import tachyon.StorageDirId; -import tachyon.StorageLevelAlias; import tachyon.TachyonURI; import tachyon.TestUtils; import tachyon.client.TachyonFS; @@ -32,11 +30,9 @@ import tachyon.master.LocalTachyonCluster; import tachyon.master.MasterInfo; import tachyon.thrift.ClientFileInfo; -import tachyon.thrift.ClientLocationInfo; import tachyon.thrift.FileAlreadyExistException; import tachyon.thrift.FileDoesNotExistException; import tachyon.thrift.InvalidPathException; -import tachyon.thrift.OutOfSpaceException; import tachyon.util.CommonUtils; /** @@ -62,7 +58,6 @@ public final void after() throws Exception { @Before public final void before() throws IOException { System.setProperty("tachyon.user.quota.unit.bytes", USER_QUOTA_UNIT_BYTES + ""); - mLocalTachyonCluster = new LocalTachyonCluster(WORKER_CAPACITY_BYTES); mLocalTachyonCluster.start(); mWorkerServiceHandler = mLocalTachyonCluster.getWorker().getWorkerServiceHandler(); @@ -101,86 +96,34 @@ public void evictionTest() throws InvalidPathException, FileAlreadyExistExceptio } @Test - public void overCapacityRequestSpaceTest() throws TException, IOException { - ClientLocationInfo locationInfo = - mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES / 10L); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo.getStorageDirId()); - Exception exception = null; - try { - mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES), exception); + public void overCapacityRequestSpaceTest() throws TException { + Assert.assertTrue(mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES / 10L)); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES * 10L)); } @Test - public void overReturnSpaceTest() throws TException, IOException { - ClientLocationInfo locationInfo = mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES / 10L); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo.getStorageDirId()); - mWorkerServiceHandler.returnSpace(1L, locationInfo.getStorageDirId(), WORKER_CAPACITY_BYTES); - Exception exception = null; - try { - mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES), exception); + public void overReturnSpaceTest() throws TException { + Assert.assertTrue(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES / 10)); + Assert.assertTrue(mWorkerServiceHandler.requestSpace(2, WORKER_CAPACITY_BYTES / 10)); + mWorkerServiceHandler.returnSpace(1, WORKER_CAPACITY_BYTES); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES)); } @Test public void returnSpaceTest() throws TException { - ClientLocationInfo locationInfo0 = mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo0.getStorageDirId()); - Exception exception = null; - try { - mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES), exception); - mWorkerServiceHandler.returnSpace(1L, locationInfo0.getStorageDirId(), WORKER_CAPACITY_BYTES); - ClientLocationInfo locationInfo2 = mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo2.getStorageDirId()); - mWorkerServiceHandler.returnSpace(2L, locationInfo2.getStorageDirId(), WORKER_CAPACITY_BYTES); - try { - mWorkerServiceHandler.requestSpace(2L, WORKER_CAPACITY_BYTES / 10); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES / 10), exception); + Assert.assertTrue(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES)); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES)); + mWorkerServiceHandler.returnSpace(1, WORKER_CAPACITY_BYTES); + Assert.assertTrue(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES)); + mWorkerServiceHandler.returnSpace(2, WORKER_CAPACITY_BYTES); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(2, WORKER_CAPACITY_BYTES / 10)); } @Test public void totalOverCapacityRequestSpaceTest() throws TException { - ClientLocationInfo locationInfo = mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES / 2); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo.getStorageDirId()); - locationInfo = mWorkerServiceHandler.requestSpace(2, WORKER_CAPACITY_BYTES / 2); - Assert.assertEquals(StorageDirId.getStorageDirId(0, StorageLevelAlias.MEM.getValue(), 0), - locationInfo.getStorageDirId()); - Exception exception = null; - try { - mWorkerServiceHandler.requestSpace(1L, WORKER_CAPACITY_BYTES / 2); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES / 2), exception); - try { - mWorkerServiceHandler.requestSpace(2L, WORKER_CAPACITY_BYTES / 2); - } catch (OutOfSpaceException e) { - exception = e; - } - Assert.assertEquals(new OutOfSpaceException( - "Failed to allocate space! requestBytes:" + WORKER_CAPACITY_BYTES / 2), exception); + Assert.assertTrue(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES / 2)); + Assert.assertTrue(mWorkerServiceHandler.requestSpace(2, WORKER_CAPACITY_BYTES / 2)); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(1, WORKER_CAPACITY_BYTES / 2)); + Assert.assertFalse(mWorkerServiceHandler.requestSpace(2, WORKER_CAPACITY_BYTES / 2)); } } diff --git a/core/src/test/java/tachyon/worker/WorkerStorageTest.java b/core/src/test/java/tachyon/worker/WorkerStorageTest.java index f854f9ddd52e..ac858276d531 100644 --- a/core/src/test/java/tachyon/worker/WorkerStorageTest.java +++ b/core/src/test/java/tachyon/worker/WorkerStorageTest.java @@ -36,11 +36,9 @@ import tachyon.client.WriteType; import tachyon.master.LocalTachyonCluster; import tachyon.thrift.NetAddress; -import tachyon.util.CommonUtils; -import tachyon.worker.hierarchy.StorageDir; /** - * Unit tests for tachyon.worker.WorkerStorage + * Unit tests for tachyon.WorkerStorage */ public class WorkerStorageTest { private static final long WORKER_CAPACITY_BYTES = 100000; @@ -86,13 +84,13 @@ private void swapoutOrphanBlocksFileTestUtil(int filesize) throws Exception { // so we need to get a fresh client to call delete mLocalTachyonCluster.getClient().delete(fid, true); - WorkerStorage ws = new WorkerStorage(mMasterAddress, mExecutorService); + WorkerStorage ws = new WorkerStorage(mMasterAddress, mWorkerDataFolder, WORKER_CAPACITY_BYTES, mExecutorService); try { ws.initialize(mWorkerAddress); String orpahnblock = ws.getUfsOrphansFolder() + TachyonURI.SEPARATOR + bid; UnderFileSystem ufs = UnderFileSystem.get(orpahnblock); - StorageDir storageDir = ws.getStorageDirByBlockId(bid); - Assert.assertFalse("Orphan block file isn't deleted from workerDataFolder", storageDir != null); + Assert.assertFalse("Orphan block file isn't deleted from workerDataFolder", new File( + mWorkerDataFolder + TachyonURI.SEPARATOR + bid).exists()); Assert.assertTrue("UFS hasn't the orphan block file ", ufs.exists(orpahnblock)); Assert.assertTrue("Orpahblock file size is changed", ufs.getFileSize(orpahnblock) == filesize); } finally { @@ -152,15 +150,13 @@ public void swapoutOrphanBlocksSmallFileTest() throws Exception { */ @Test public void unknownBlockFilesTest() throws Exception { - String dirPath = System.getProperty("tachyon.worker.hierarchystore.level0.dirs.path"); - String dataFolder = CommonUtils.concat(dirPath, mWorkerDataFolder); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Wrong file name: xyz"); mLocalTachyonCluster.stopWorker(); // try a non-numerical file name - File unknownFile = new File(dataFolder + TachyonURI.SEPARATOR + "xyz"); + File unknownFile = new File(mWorkerDataFolder + TachyonURI.SEPARATOR + "xyz"); unknownFile.createNewFile(); - WorkerStorage ws = new WorkerStorage(mMasterAddress, mExecutorService); + WorkerStorage ws = new WorkerStorage(mMasterAddress, mWorkerDataFolder, WORKER_CAPACITY_BYTES, mExecutorService); try { ws.initialize(mWorkerAddress); } finally { diff --git a/core/src/test/java/tachyon/worker/hierarchy/AllocateStrategyTest.java b/core/src/test/java/tachyon/worker/hierarchy/AllocateStrategyTest.java index 3bc1758748ec..2b708498974c 100644 --- a/core/src/test/java/tachyon/worker/hierarchy/AllocateStrategyTest.java +++ b/core/src/test/java/tachyon/worker/hierarchy/AllocateStrategyTest.java @@ -24,14 +24,14 @@ import tachyon.TestUtils; import tachyon.UnderFileSystem; -import tachyon.worker.BlockHandler; +import tachyon.client.BlockHandler; import tachyon.master.BlockInfo; import tachyon.util.CommonUtils; public class AllocateStrategyTest { private final StorageDir[] mStorageDirs = new StorageDir[3]; - private static final long USER_ID = 1; - private static final long[] CAPACITIES = new long[] {1000, 1100, 1200}; + private final long mUserId = 1; + private final long[] mCapacities = new long[] {1000, 1100, 1200}; @Before public final void before() throws IOException { @@ -41,9 +41,9 @@ public final void before() throws IOException { String[] dirPaths = "/dir1,/dir2,/dir3".split(","); for (int i = 0; i < 3; i++) { mStorageDirs[i] = - new StorageDir(i + 1, workerDirFolder + dirPaths[i], CAPACITIES[i], "/data", "/user", + new StorageDir(i + 1, workerDirFolder + dirPaths[i], mCapacities[i], "/data", "/user", null); - initializeStorageDir(mStorageDirs[i], USER_ID); + initializeStorageDir(mStorageDirs[i], mUserId); } } @@ -51,34 +51,34 @@ private void createBlockFile(StorageDir dir, long blockId, int blockSize) throws byte[] buf = TestUtils.getIncreasingByteArray(blockSize); BlockHandler bhSrc = - BlockHandler.get(dir.getUserTempFilePath(USER_ID, blockId)); + BlockHandler.get(dir.getUserTempFilePath(mUserId, blockId)); try { bhSrc.append(0, ByteBuffer.wrap(buf)); } finally { bhSrc.close(); } - dir.requestSpace(USER_ID, blockSize); - dir.cacheBlock(USER_ID, blockId); + dir.requestSpace(mUserId, blockSize); + dir.cacheBlock(mUserId, blockId); } @Test public void AllocateMaxFreeTest() throws IOException { AllocateStrategy allocator = AllocateStrategies.getAllocateStrategy(AllocateStrategyType.MAX_FREE); - StorageDir storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + StorageDir storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(3, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(1, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(2, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(2, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(1, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(3, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 1000); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 1000); Assert.assertEquals(null, storageDir); boolean fitIn = allocator.fitInPossible(mStorageDirs, 1200); Assert.assertEquals(true, fitIn); - mStorageDirs[2].lockBlock(BlockInfo.computeBlockId(1, 0), USER_ID); + mStorageDirs[2].lockBlock(BlockInfo.computeBlockId(1, 0), mUserId); fitIn = allocator.fitInPossible(mStorageDirs, 1200); Assert.assertEquals(false, fitIn); } @@ -87,20 +87,20 @@ public void AllocateMaxFreeTest() throws IOException { public void AllocateRoundRobinTest() throws IOException { AllocateStrategy allocator = AllocateStrategies.getAllocateStrategy(AllocateStrategyType.ROUND_ROBIN); - StorageDir storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + StorageDir storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(1, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(1, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(2, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(2, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(3, storageDir.getStorageDirId()); createBlockFile(storageDir, BlockInfo.computeBlockId(3, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 1000); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 1000); Assert.assertEquals(null, storageDir); boolean fitIn = allocator.fitInPossible(mStorageDirs, 1200); Assert.assertEquals(true, fitIn); - mStorageDirs[2].lockBlock(BlockInfo.computeBlockId(3, 0), USER_ID); + mStorageDirs[2].lockBlock(BlockInfo.computeBlockId(3, 0), mUserId); fitIn = allocator.fitInPossible(mStorageDirs, 1200); Assert.assertEquals(false, fitIn); } @@ -109,16 +109,16 @@ public void AllocateRoundRobinTest() throws IOException { public void AllocateRandomTest() throws IOException { AllocateStrategy allocator = AllocateStrategies.getAllocateStrategy(AllocateStrategyType.RANDOM); - StorageDir storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + StorageDir storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(true, storageDir != null); createBlockFile(storageDir, BlockInfo.computeBlockId(1, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(true, storageDir != null); createBlockFile(storageDir, BlockInfo.computeBlockId(2, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 300); Assert.assertEquals(true, storageDir != null); createBlockFile(storageDir, BlockInfo.computeBlockId(3, 0), 300); - storageDir = allocator.getStorageDir(mStorageDirs, USER_ID, 1300); + storageDir = allocator.getStorageDir(mStorageDirs, mUserId, 1300); Assert.assertEquals(null, storageDir); boolean fitIn = allocator.fitInPossible(mStorageDirs, 1200); Assert.assertEquals(true, fitIn); diff --git a/core/src/test/java/tachyon/worker/hierarchy/EvictStrategyTest.java b/core/src/test/java/tachyon/worker/hierarchy/EvictStrategyTest.java index 1834829490c1..4bcff4f30f7d 100644 --- a/core/src/test/java/tachyon/worker/hierarchy/EvictStrategyTest.java +++ b/core/src/test/java/tachyon/worker/hierarchy/EvictStrategyTest.java @@ -28,17 +28,17 @@ import tachyon.Pair; import tachyon.TestUtils; import tachyon.UnderFileSystem; +import tachyon.client.BlockHandler; import tachyon.master.BlockInfo; import tachyon.util.CommonUtils; -import tachyon.worker.BlockHandler; import tachyon.worker.eviction.EvictLRU; import tachyon.worker.eviction.EvictPartialLRU; import tachyon.worker.eviction.EvictStrategy; public class EvictStrategyTest { private final StorageDir[] mStorageDirs = new StorageDir[3]; - private static final long USER_ID = 1; - private static final long CAPACITY = 1000; + private final long mUserId = 1; + private final long mCapacity = 1000; @Before public final void before() throws IOException { @@ -48,8 +48,8 @@ public final void before() throws IOException { String[] dirPaths = "/dir1,/dir2,/dir3".split(","); for (int i = 0; i < 3; i++) { mStorageDirs[i] = - new StorageDir(i + 1, workerDirFolder + dirPaths[i], CAPACITY, "/data", "/user", null); - initializeStorageDir(mStorageDirs[i], USER_ID); + new StorageDir(i + 1, workerDirFolder + dirPaths[i], mCapacity, "/data", "/user", null); + initializeStorageDir(mStorageDirs[i], mUserId); } } @@ -57,14 +57,14 @@ private void createBlockFile(StorageDir dir, long blockId, int blockSize) throws byte[] buf = TestUtils.getIncreasingByteArray(blockSize); BlockHandler bhSrc = - BlockHandler.get(dir.getUserTempFilePath(USER_ID, blockId)); + BlockHandler.get(dir.getUserTempFilePath(mUserId, blockId)); try { bhSrc.append(0, ByteBuffer.wrap(buf)); } finally { bhSrc.close(); } - dir.requestSpace(USER_ID, blockSize); - dir.cacheBlock(USER_ID, blockId); + dir.requestSpace(mUserId, blockSize); + dir.cacheBlock(mUserId, blockId); } @Test diff --git a/core/src/test/java/tachyon/worker/hierarchy/HierarchyStoreTest.java b/core/src/test/java/tachyon/worker/hierarchy/HierarchyStoreTest.java deleted file mode 100644 index 5dd9ba36f891..000000000000 --- a/core/src/test/java/tachyon/worker/hierarchy/HierarchyStoreTest.java +++ /dev/null @@ -1,115 +0,0 @@ -package tachyon.worker.hierarchy; - -import java.io.IOException; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import tachyon.TestUtils; -import tachyon.client.InStream; -import tachyon.client.ReadType; -import tachyon.client.TachyonFS; -import tachyon.client.TachyonFile; -import tachyon.client.WriteType; -import tachyon.conf.WorkerConf; -import tachyon.master.LocalTachyonCluster; -import tachyon.util.CommonUtils; - -/** - * Unit tests for tachyon.worker.StorageTier. - */ -public class HierarchyStoreTest { - private static final int MEM_CAPACITY_BYTES = 1000; - private static final int DISK_CAPACITY_BYTES = 10000; - private static final int USER_QUOTA_UNIT_BYTES = 100; - private static final int WORKER_TO_MASTER_HEARTBEAT_INTERVAL_MS - = WorkerConf.get().TO_MASTER_HEARTBEAT_INTERVAL_MS; - - private LocalTachyonCluster mLocalTachyonCluster = null; - private TachyonFS mTFS = null; - - @After - public final void after() throws Exception { - mLocalTachyonCluster.stop(); - System.clearProperty("tachyon.user.quota.unit.bytes"); - System.clearProperty("tachyon.worker.hierarchystore.level.max"); - } - - @Before - public final void before() throws IOException { - System.setProperty("tachyon.user.quota.unit.bytes", USER_QUOTA_UNIT_BYTES + ""); - mLocalTachyonCluster = new LocalTachyonCluster(MEM_CAPACITY_BYTES); - System.setProperty("tachyon.worker.hierarchystore.level.max", 2 + ""); - System.setProperty("tachyon.worker.hierarchystore.level1.alias", "HDD"); - System.setProperty("tachyon.worker.hierarchystore.level1.dirs.path", "/disk1" + "," + "/disk2"); - System.setProperty("tachyon.worker.hierarchystore.level1.dirs.quota", DISK_CAPACITY_BYTES + ""); - mLocalTachyonCluster.start(); - mTFS = mLocalTachyonCluster.getClient(); - } - - @Test - public void blockEvict() throws IOException, InterruptedException { - int fileId1 = - TestUtils.createByteFile(mTFS, "/root/test1", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 6); - int fileId2 = - TestUtils.createByteFile(mTFS, "/root/test2", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 6); - int fileId3 = - TestUtils.createByteFile(mTFS, "/root/test3", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 6); - - TachyonFile file1 = mTFS.getFile(fileId1); - TachyonFile file2 = mTFS.getFile(fileId2); - TachyonFile file3 = mTFS.getFile(fileId3); - - Assert.assertEquals(file1.isInMemory(), true); - Assert.assertEquals(file2.isInMemory(), true); - Assert.assertEquals(file3.isInMemory(), true); - - int fileId4 = - TestUtils.createByteFile(mTFS, "/root/test4", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 2); - int fileId5 = - TestUtils.createByteFile(mTFS, "/root/test5", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 2); - - CommonUtils.sleepMs(null, WORKER_TO_MASTER_HEARTBEAT_INTERVAL_MS); - TachyonFile file4 = mTFS.getFile(fileId4); - TachyonFile file5 = mTFS.getFile(fileId5); - - Assert.assertEquals(file1.isInMemory(), false); - Assert.assertEquals(file2.isInMemory(), false); - Assert.assertEquals(file3.isInMemory(), false); - Assert.assertEquals(file4.isInMemory(), true); - Assert.assertEquals(file5.isInMemory(), true); - } - - @Test - public void promoteBlock() throws IOException, InterruptedException { - int fileId1 = - TestUtils.createByteFile(mTFS, "/root/test1", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 6); - int fileId2 = - TestUtils.createByteFile(mTFS, "/root/test2", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 2); - int fileId3 = - TestUtils.createByteFile(mTFS, "/root/test3", WriteType.TRY_CACHE, MEM_CAPACITY_BYTES / 2); - - CommonUtils.sleepMs(null, WORKER_TO_MASTER_HEARTBEAT_INTERVAL_MS); - TachyonFile file1 = mTFS.getFile(fileId1); - TachyonFile file2 = mTFS.getFile(fileId2); - TachyonFile file3 = mTFS.getFile(fileId3); - - Assert.assertEquals(false, file1.isInMemory()); - Assert.assertEquals(true, file2.isInMemory()); - Assert.assertEquals(true, file3.isInMemory()); - - InStream is = file1.getInStream(ReadType.CACHE_PROMOTE); - byte[] buf = new byte[MEM_CAPACITY_BYTES / 6]; - int len = is.read(buf); - - CommonUtils.sleepMs(null, WORKER_TO_MASTER_HEARTBEAT_INTERVAL_MS); - Assert.assertEquals(MEM_CAPACITY_BYTES / 6, len); - Assert.assertEquals(true, file1.isInMemory()); - Assert.assertEquals(false, file2.isInMemory()); - Assert.assertEquals(true, file3.isInMemory()); - Assert.assertEquals(MEM_CAPACITY_BYTES / 6 + MEM_CAPACITY_BYTES, - mLocalTachyonCluster.getMasterInfo().getUsedBytes()); - } -} diff --git a/core/src/test/java/tachyon/worker/hierarchy/StorageDirTest.java b/core/src/test/java/tachyon/worker/hierarchy/StorageDirTest.java index 539999bf298c..97eea39eb173 100644 --- a/core/src/test/java/tachyon/worker/hierarchy/StorageDirTest.java +++ b/core/src/test/java/tachyon/worker/hierarchy/StorageDirTest.java @@ -24,26 +24,26 @@ import tachyon.TestUtils; import tachyon.UnderFileSystem; +import tachyon.client.BlockHandler; import tachyon.thrift.InvalidPathException; import tachyon.util.CommonUtils; -import tachyon.worker.BlockHandler; public class StorageDirTest { private StorageDir mSrcDir; private StorageDir mDstDir; - private static final long USER_ID = 1; - private static final long CAPACITY = 1000; + private final long mUserId = 1; + private final long mCapacity = 1000; @Before public final void before() throws IOException, InvalidPathException { String tachyonHome = File.createTempFile("Tachyon", "").getAbsoluteFile() + "U" + System.currentTimeMillis(); String workerDirFolder = tachyonHome + "/ramdisk"; - mSrcDir = new StorageDir(1, workerDirFolder + "/src", CAPACITY, "/data", "/user", null); - mDstDir = new StorageDir(2, workerDirFolder + "/dst", CAPACITY, "/data", "/user", null); + mSrcDir = new StorageDir(1, workerDirFolder + "/src", mCapacity, "/data", "/user", null); + mDstDir = new StorageDir(2, workerDirFolder + "/dst", mCapacity, "/data", "/user", null); - initializeStorageDir(mSrcDir, USER_ID); - initializeStorageDir(mDstDir, USER_ID); + initializeStorageDir(mSrcDir, mUserId); + initializeStorageDir(mDstDir, mUserId); } @Test @@ -53,31 +53,31 @@ public void copyBlockTest() throws IOException { createBlockFile(mSrcDir, blockId, blockSize); Assert.assertTrue(mSrcDir.containsBlock(blockId)); - Assert.assertEquals(CAPACITY - blockSize, mSrcDir.getAvailableBytes()); + Assert.assertEquals(mCapacity - blockSize, mSrcDir.getAvailableBytes()); Assert.assertEquals(blockSize, mSrcDir.getBlockSize(blockId)); - boolean requestDst = mDstDir.requestSpace(USER_ID, blockSize); + boolean requestDst = mDstDir.requestSpace(mUserId, blockSize); Assert.assertTrue(requestDst); mSrcDir.copyBlock(blockId, mDstDir); Assert.assertTrue(mDstDir.containsBlock(blockId)); - Assert.assertEquals(CAPACITY - blockSize, mDstDir.getAvailableBytes()); + Assert.assertEquals(mCapacity - blockSize, mDstDir.getAvailableBytes()); Assert.assertEquals(blockSize, mDstDir.getBlockSize(blockId)); mSrcDir.deleteBlock(blockId); Assert.assertFalse(mSrcDir.containsBlock(blockId)); - Assert.assertEquals(CAPACITY, mSrcDir.getAvailableBytes()); + Assert.assertEquals(mCapacity, mSrcDir.getAvailableBytes()); } private void createBlockFile(StorageDir dir, long blockId, int blockSize) throws IOException { byte[] buf = TestUtils.getIncreasingByteArray(blockSize); BlockHandler bhSrc = - BlockHandler.get(CommonUtils.concat(dir.getUserTempFilePath(USER_ID, blockId))); + BlockHandler.get(CommonUtils.concat(dir.getUserTempFilePath(mUserId, blockId))); try { bhSrc.append(0, ByteBuffer.wrap(buf)); } finally { bhSrc.close(); } - dir.requestSpace(USER_ID, blockSize); - dir.cacheBlock(USER_ID, blockId); + dir.requestSpace(mUserId, blockSize); + dir.cacheBlock(mUserId, blockId); } @Test @@ -105,9 +105,9 @@ public void lockBlockTest() throws IOException { long blockId = 100; createBlockFile(mSrcDir, blockId, 500); - mSrcDir.lockBlock(blockId, USER_ID); + mSrcDir.lockBlock(blockId, mUserId); Assert.assertTrue(mSrcDir.isBlockLocked(blockId)); - mSrcDir.unlockBlock(blockId, USER_ID); + mSrcDir.unlockBlock(blockId, mUserId); Assert.assertFalse(mSrcDir.isBlockLocked(blockId)); } @@ -117,7 +117,7 @@ public void moveBlockTest() throws IOException { int blockSize = 500; createBlockFile(mSrcDir, blockId, blockSize); - mDstDir.requestSpace(USER_ID, blockSize); + mDstDir.requestSpace(mUserId, blockSize); mSrcDir.moveBlock(blockId, mDstDir); Assert.assertFalse(mSrcDir.containsBlock(blockId)); Assert.assertTrue(mDstDir.containsBlock(blockId)); @@ -126,13 +126,13 @@ public void moveBlockTest() throws IOException { @Test public void requestSpaceTest() { - boolean requestSrc = mSrcDir.requestSpace(USER_ID, CAPACITY / 2); + boolean requestSrc = mSrcDir.requestSpace(mUserId, mCapacity / 2); Assert.assertTrue(requestSrc); - requestSrc = mSrcDir.requestSpace(USER_ID, CAPACITY / 2 + 1); + requestSrc = mSrcDir.requestSpace(mUserId, mCapacity / 2 + 1); Assert.assertFalse(requestSrc); - Assert.assertEquals(CAPACITY / 2, mSrcDir.getUsedBytes()); - Assert.assertEquals(CAPACITY / 2, mSrcDir.getAvailableBytes()); - mSrcDir.returnSpace(USER_ID, CAPACITY / 2); - Assert.assertEquals(CAPACITY, mSrcDir.getAvailableBytes()); + Assert.assertEquals(mCapacity / 2, mSrcDir.getUsedBytes()); + Assert.assertEquals(mCapacity / 2, mSrcDir.getAvailableBytes()); + mSrcDir.returnSpace(mUserId, mCapacity / 2); + Assert.assertEquals(mCapacity, mSrcDir.getAvailableBytes()); } } diff --git a/core/src/test/java/tachyon/worker/hierarchy/StorageTierTest.java b/core/src/test/java/tachyon/worker/hierarchy/StorageTierTest.java index 67e483985032..a8f83d5b936e 100644 --- a/core/src/test/java/tachyon/worker/hierarchy/StorageTierTest.java +++ b/core/src/test/java/tachyon/worker/hierarchy/StorageTierTest.java @@ -17,9 +17,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; import junit.framework.Assert; @@ -30,14 +28,14 @@ import tachyon.StorageLevelAlias; import tachyon.TestUtils; import tachyon.UnderFileSystem; +import tachyon.client.BlockHandler; import tachyon.conf.WorkerConf; import tachyon.thrift.InvalidPathException; import tachyon.util.CommonUtils; -import tachyon.worker.BlockHandler; public class StorageTierTest { - private static final long USER_ID = 1; + private final long mUserId = 1; private StorageTier[] mStorageTiers; @@ -50,7 +48,6 @@ public final void after() throws Exception { System.clearProperty("tachyon.worker.hierarchystore.level1.alias"); System.clearProperty("tachyon.worker.hierarchystore.level1.dirs.path"); System.clearProperty("tachyon.worker.hierarchystore.level1.dirs.quota"); - WorkerConf.clear(); } @Before @@ -76,7 +73,7 @@ public final void before() throws IOException, InvalidPathException { for (int i = 0; i < dirPaths.length; i ++) { dirPaths[i] = dirPaths[i].trim(); } - StorageLevelAlias storageLevelAlias = WorkerConf.get().STORAGE_LEVEL_ALIAS[level]; + StorageLevelAlias Alias = WorkerConf.get().STORAGE_LEVEL_ALIAS[level]; String[] strDirCapacities = WorkerConf.get().STORAGE_TIER_DIR_QUOTA[level].split(","); long[] dirCapacities = new long[dirPaths.length]; for (int i = 0, j = 0; i < dirPaths.length; i ++) { @@ -87,12 +84,11 @@ public final void before() throws IOException, InvalidPathException { } } StorageTier curTier = - new StorageTier(level, storageLevelAlias, dirPaths, dirCapacities, "/data", "/user", - nextTier, null); + new StorageTier(level, Alias, dirPaths, dirCapacities, "/data", "/user", nextTier, null); mStorageTiers[level] = curTier; curTier.initialize(); for (StorageDir dir : curTier.getStorageDirs()) { - initializeStorageDir(dir, USER_ID); + initializeStorageDir(dir, mUserId); } nextTier = curTier; } @@ -101,21 +97,19 @@ public final void before() throws IOException, InvalidPathException { private void createBlockFile(StorageDir dir, long blockId, int blockSize) throws IOException { byte[] buf = TestUtils.getIncreasingByteArray(blockSize); BlockHandler bhSrc = - BlockHandler.get(CommonUtils.concat(dir.getUserTempFilePath(USER_ID, blockId))); + BlockHandler.get(CommonUtils.concat(dir.getUserTempFilePath(mUserId, blockId))); try { bhSrc.append(0, ByteBuffer.wrap(buf)); } finally { bhSrc.close(); } - dir.cacheBlock(USER_ID, blockId); + dir.cacheBlock(mUserId, blockId); } @Test public void getStorageDirTest() throws IOException { long blockId = 1; - List removedBlockIds = new ArrayList(); - StorageDir dir = mStorageTiers[0].requestSpace(USER_ID, 100, new HashSet(), - removedBlockIds); + StorageDir dir = mStorageTiers[0].requestSpace(mUserId, 100, new HashSet()); createBlockFile(dir, blockId, 100); StorageDir dir1 = mStorageTiers[0].getStorageDirByBlockId(1); Assert.assertEquals(dir, dir1); @@ -142,27 +136,22 @@ public void isLastTierTest() { @Test public void requestSpaceTest() throws IOException { long blockId = 1; - List removedBlockIds = new ArrayList(); Assert.assertEquals(1000, mStorageTiers[0].getCapacityBytes()); Assert.assertEquals(8000, mStorageTiers[1].getCapacityBytes()); - StorageDir dir = mStorageTiers[0].requestSpace(USER_ID, 500, new HashSet(), - removedBlockIds); + StorageDir dir = mStorageTiers[0].requestSpace(mUserId, 500, new HashSet()); Assert.assertEquals(mStorageTiers[0].getStorageDirs()[0], dir); Assert.assertEquals(500, dir.getAvailableBytes()); Assert.assertEquals(500, dir.getUsedBytes()); - StorageDir dir1 = mStorageTiers[0].requestSpace(USER_ID, 501, new HashSet(), - removedBlockIds); + StorageDir dir1 = mStorageTiers[0].requestSpace(mUserId, 501, new HashSet()); Assert.assertEquals(null, dir1); createBlockFile(dir, blockId, 500); - boolean request = mStorageTiers[0].requestSpace(dir, USER_ID, 501, new HashSet(), - removedBlockIds); + boolean request = mStorageTiers[0].requestSpace(dir, mUserId, 501, new HashSet()); Assert.assertEquals(true, request); Assert.assertEquals(499, dir.getAvailableBytes()); Assert.assertEquals(501, dir.getUsedBytes()); Assert.assertTrue(mStorageTiers[1].containsBlock(blockId)); Assert.assertEquals(500, mStorageTiers[1].getUsedBytes()); - request = mStorageTiers[0].requestSpace(dir, USER_ID, 500, new HashSet(), - removedBlockIds); + request = mStorageTiers[0].requestSpace(dir, mUserId, 500, new HashSet()); Assert.assertEquals(false, request); } } diff --git a/core/src/thrift/tachyon.thrift b/core/src/thrift/tachyon.thrift index 3f5e3d5458cf..1dd83044c1a6 100644 --- a/core/src/thrift/tachyon.thrift +++ b/core/src/thrift/tachyon.thrift @@ -15,7 +15,6 @@ struct ClientBlockInfo { 2: i64 offset 3: i64 length 4: list locations - 5: map storageDirIds } struct ClientWorkerInfo { @@ -74,16 +73,11 @@ struct Command { 2: list mData } -struct ClientLocationInfo { - 1: i64 storageDirId - 2: string path //TODO add classname to make it pluggable -} - exception BlockInfoException { 1: string message } -exception OutOfSpaceException { +exception OutOfMemoryForPinFileException { 1: string message } @@ -143,14 +137,13 @@ service MasterService { * @return value rv % 100,000 is really workerId, rv / 1000,000 is master started time. */ i64 worker_register(1: NetAddress workerNetAddress, 2: i64 totalBytes, 3: i64 usedBytes, - 4: map> currentBlocks) + 4: list currentBlocks) throws (1: BlockInfoException e) - Command worker_heartbeat(1: i64 workerId, 2: i64 usedBytes, 3: list removedBlockIds, - 4: map> evictedBlockIds) + Command worker_heartbeat(1: i64 workerId, 2: i64 usedBytes, 3: list removedBlocks) throws (1: BlockInfoException e) - void worker_cacheBlock(1: i64 workerId, 2: i64 workerUsedBytes, 3: i64 storageDirId, 4: i64 blockId, 5: i64 length) + void worker_cacheBlock(1: i64 workerId, 2: i64 workerUsedBytes, 3: i64 blockId, 4: i64 length) throws (1: FileDoesNotExistException eP, 2: SuspectedFileSizeException eS, 3: BlockInfoException eB) set worker_getPinIdList() @@ -255,7 +248,7 @@ service MasterService { } service WorkerService { - void accessBlock(1: i64 storageDirId, 2: i64 blockId) + void accessBlock(1: i64 blockId) void addCheckpoint(1: i64 userId, 2: i32 fileId) throws (1: FileDoesNotExistException eP, 2: SuspectedFileSizeException eS, @@ -264,30 +257,23 @@ service WorkerService { bool asyncCheckpoint(1: i32 fileId) throws (1: TachyonException e) - void cacheBlock(1: i64 userId, 2: i64 storageDirId, 3: i64 blockId) + void cacheBlock(1: i64 userId, 2: i64 blockId) throws (1: FileDoesNotExistException eP, 2: SuspectedFileSizeException eS, 3: BlockInfoException eB) - ClientLocationInfo getLocalBlockLocation(1: i64 blockId) - throws (1: FileDoesNotExistException eP) + string getDataFolder() - string getUserLocalTempFolder(1: i64 userId 2: i64 storageDirId) + string getUserTempFolder(1: i64 userId) string getUserUfsTempFolder(1: i64 userId) - ClientLocationInfo lockBlock(1: i64 blockId 2: i64 userId) // Lock the file in memory while the user is reading it. - throws (1: FileDoesNotExistException eP) - - bool promoteBlock(1: i64 userId, 2: i64 blockId) - - void returnSpace(1: i64 userId, 2: i64 storageDirId 3: i64 returnedBytes) + void lockBlock(1: i64 blockId, 2: i64 userId) // Lock the file in memory while the user is reading it. - ClientLocationInfo requestSpace(1: i64 userId, 2: i64 requestBytes) - throws (1: OutOfSpaceException eP) + void returnSpace(1: i64 userId, 2: i64 returnedBytes) - bool requestSpaceInPlace(1: i64 userId, 2: i64 storageDirId, 3: i64 requestBytes) + bool requestSpace(1: i64 userId, 2: i64 requestBytes) // Should change this to return i64, means how much space to grant. - bool unlockBlock(1: i64 blockId 2: i64 userId) // unlock the file + void unlockBlock(1: i64 blockId, 2: i64 userId) // unlock the file void userHeartbeat(1: i64 userId) // Local user send heartbeat to local worker to keep its temp folder. }