@Override public RpcFuture<RpcGetDataNode> getDataNode(DataNodeInfo dnInfo) throws Exception { // LOG.info("issuing get datanode on index " + getDataNodeIndex); RpcFuture<RpcGetDataNode> res = connections[getDataNodeIndex].getDataNode(dnInfo); getDataNodeIndex = (getDataNodeIndex + 1) % connections.length; return res; }
@Override public RpcFuture<RpcVoid> setBlock(BlockInfo blockInfo) throws Exception { // LOG.info("issuing set block on index " + setBlockIndex); RpcFuture<RpcVoid> res = connections[setBlockIndex].setBlock(blockInfo); setBlockIndex = (setBlockIndex + 1) % connections.length; return res; }
@Override public void close() throws Exception { for (RpcConnection connection : connections){ connection.close(); } }
public Upcoming<CrailNode> lookup(String path) throws Exception { FileName name = new FileName(path); if (CrailConstants.DEBUG){ LOG.info("lookupDirectory: path " + path); } RpcFuture<RpcGetFile> fileRes = rpcConnection.getFile(name, false); return new LookupNodeFuture(this, path, fileRes); }
@Override public RpcFuture<RpcGetBlock> getBlock(long fd, long token, long position, long capacity) throws IOException { int index = computeIndex(fd); // LOG.info("issuing get block for fd [" + fd + "], on index " + index); return connections[index].getBlock(fd, token, position, capacity); }
@Override public RpcFuture<RpcPing> pingNameNode() throws Exception { return connections[0].pingNameNode(); }
@Override public RpcFuture<RpcGetLocation> getLocation(FileName fileName, long position) throws IOException { int index = computeIndex(fileName.getComponent(0)); // LOG.info("issuing get location for filename [" + fileName.toString() + "], on index " + index); return connections[index].getLocation(fileName, position); }
@Override public RpcFuture<RpcCreateFile> createFile(FileName filename, CrailNodeType type, int storageClass, int locationClass) throws IOException { int index = computeIndex(filename.getComponent(0)); // LOG.info("issuing create file for filename [" + filename.toString() + "], on index " + index); return connections[index].createFile(filename, type, storageClass, locationClass); }
public Upcoming<CrailNode> delete(String path, boolean recursive) throws Exception { FileName name = new FileName(path); if (CrailConstants.DEBUG){ LOG.info("delete: name " + path + ", recursive " + recursive); } RpcFuture<RpcDeleteFile> fileRes = rpcConnection.removeFile(name, recursive); return new DeleteNodeFuture(this, path, recursive, fileRes); }
@Override public RpcFuture<RpcVoid> dumpNameNode() throws Exception { return connections[0].dumpNameNode(); }
@Override public RpcFuture<RpcGetFile> getFile(FileName filename, boolean writeable) throws IOException { int index = computeIndex(filename.getComponent(0)); // LOG.info("issuing get file for filename [" + filename.toString() + "], on index " + index); return connections[index].getFile(filename, writeable); }
final void prefetchMetadata() throws Exception { long key = CoreSubOperation.createKey(fileInfo.getFd(), position); if (blockCache.containsKey(key)){ return; } if (nextBlockCache.containsKey(key)){ return; } this.syncedCapacity = fileInfo.getCapacity(); RpcFuture<RpcGetBlock> nextBlock = namenodeClientRpc.getBlock(fileInfo.getFd(), fileInfo.getToken(), position, syncedCapacity); nextBlock.setPrefetched(true); nextBlockCache.put(key, nextBlock); this.ioStats.incPrefetchedOps(); }
public void ping() throws Exception { RpcPing pingRes = rpcConnection.pingNameNode().get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS); if (pingRes.getError() != RpcErrors.ERR_OK) { LOG.info("Ping: " + RpcErrors.messages[pingRes.getError()]); throw new IOException(RpcErrors.messages[pingRes.getError()]); } }
RpcGetLocation getLocationRes = rpcConnection.getLocation(name, current).get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS); if (getLocationRes.getError() != RpcErrors.ERR_OK) { LOG.info("location: " + RpcErrors.messages[getLocationRes.getError()]);
public Upcoming<CrailNode> create(String path, CrailNodeType type, CrailStorageClass storageClass, CrailLocationClass locationClass) throws Exception { FileName name = new FileName(path); if (CrailConstants.DEBUG){ LOG.info("createNode: name " + path + ", type " + type + ", storageAffinity " + storageClass + ", locationAffinity " + locationClass); } RpcFuture<RpcCreateFile> fileRes = rpcConnection.createFile(name, type, storageClass.value(), locationClass.value()); return new CreateNodeFuture(this, path, type, fileRes); }
@Override public RpcFuture<RpcDeleteFile> removeFile(FileName filename, boolean recursive) throws IOException { int index = computeIndex(filename.getComponent(0)); // LOG.info("issuing remove file for filename [" + filename.toString() + "], on index " + index); return connections[index].removeFile(filename, recursive); }
public void dumpNameNode() throws Exception { rpcConnection.dumpNameNode().get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS); }
public DataNodeStatistics getDataNode() throws Exception{ InetSocketAddress inetAddress = serverAddress; DataNodeInfo dnInfo = new DataNodeInfo(storageType, storageClass.value(), locationClass.value(), inetAddress.getAddress().getAddress(), inetAddress.getPort()); return this.rpcConnection.getDataNode(dnInfo).get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS).getStatistics(); } }
public void setBlock(long lba, long addr, int length, int key) throws Exception { InetSocketAddress inetAddress = serverAddress; DataNodeInfo dnInfo = new DataNodeInfo(storageType, storageClass.value(), locationClass.value(), inetAddress.getAddress().getAddress(), inetAddress.getPort()); BlockInfo blockInfo = new BlockInfo(dnInfo, lba, addr, length, key); RpcVoid res = rpcConnection.setBlock(blockInfo).get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS); if (res.getError() != RpcErrors.ERR_OK){ LOG.info("setBlock: " + RpcErrors.messages[res.getError()]); throw new IOException("setBlock: " + RpcErrors.messages[res.getError()]); } }
public DirectoryInputStream _listEntries(String name, boolean randomize) throws Exception { FileName directory = new FileName(name); if (CrailConstants.DEBUG){ LOG.info("getDirectoryList: " + name); } RpcGetFile fileRes = rpcConnection.getFile(directory, false).get(CrailConstants.RPC_TIMEOUT, TimeUnit.MILLISECONDS); if (fileRes.getError() != RpcErrors.ERR_OK) { LOG.info("getDirectoryList: " + RpcErrors.messages[fileRes.getError()]); throw new FileNotFoundException(RpcErrors.messages[fileRes.getError()]); } FileInfo dirInfo = fileRes.getFile(); if (!dirInfo.getType().isContainer()){ LOG.info("getDirectoryList: " + RpcErrors.messages[RpcErrors.ERR_FILE_IS_NOT_DIR]); throw new FileNotFoundException(RpcErrors.messages[RpcErrors.ERR_FILE_IS_NOT_DIR]); } CoreDirectory dirFile = new CoreDirectory(this, dirInfo, name); DirectoryInputStream inputStream = dirFile.getDirectoryInputStream(randomize); return inputStream; }