HDFS-5631. Change BlockMetadataHeader.readHeader(..), ChunkChecksum class and constructor to public; and fix FsDatasetSpi to use generic type instead of FsVolumeImpl. Contributed by David Powell and Joe Pallas
This commit is contained in:
parent
7fc1f2f5cf
commit
4a4450836c
|
@ -136,6 +136,10 @@ Trunk (Unreleased)
|
|||
HDFS-7591. hdfs classpath command should support same options as hadoop
|
||||
classpath (Varun Saxena via Arpit Agarwal)
|
||||
|
||||
HDFS-5631. Change BlockMetadataHeader.readHeader(..), ChunkChecksum
|
||||
class and constructor to public; and fix FsDatasetSpi to use generic type
|
||||
instead of FsVolumeImpl. (David Powell and Joe Pallas via szetszwo)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -162,7 +162,7 @@ public class BlockMetadataHeader {
|
|||
* The current file position will be altered by this method.
|
||||
* If an error occurs, the file is <em>not</em> closed.
|
||||
*/
|
||||
static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
|
||||
public static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
|
||||
byte[] buf = new byte[getHeaderSize()];
|
||||
raf.seek(0);
|
||||
raf.readFully(buf, 0, buf.length);
|
||||
|
|
|
@ -26,12 +26,12 @@ package org.apache.hadoop.hdfs.server.datanode;
|
|||
* the checksum applies for the last chunk, or bytes 512 - 1023
|
||||
*/
|
||||
|
||||
class ChunkChecksum {
|
||||
public class ChunkChecksum {
|
||||
private final long dataLength;
|
||||
// can be null if not available
|
||||
private final byte[] checksum;
|
||||
|
||||
ChunkChecksum(long dataLength, byte[] checksum) {
|
||||
public ChunkChecksum(long dataLength, byte[] checksum) {
|
||||
this.dataLength = dataLength;
|
||||
this.checksum = checksum;
|
||||
}
|
||||
|
|
|
@ -503,7 +503,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
|
|||
* Callback from RamDiskAsyncLazyPersistService upon async lazy persist task end
|
||||
*/
|
||||
public void onCompleteLazyPersist(String bpId, long blockId,
|
||||
long creationTime, File[] savedFiles, FsVolumeImpl targetVolume);
|
||||
long creationTime, File[] savedFiles, V targetVolume);
|
||||
|
||||
/**
|
||||
* Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail
|
||||
|
|
|
@ -229,14 +229,15 @@ class RamDiskAsyncLazyPersistService {
|
|||
@Override
|
||||
public void run() {
|
||||
boolean succeeded = false;
|
||||
final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
|
||||
try {
|
||||
// No FsDatasetImpl lock for the file copy
|
||||
File targetFiles[] = FsDatasetImpl.copyBlockFiles(
|
||||
blockId, genStamp, metaFile, blockFile, lazyPersistDir, true);
|
||||
|
||||
// Lock FsDataSetImpl during onCompleteLazyPersist callback
|
||||
datanode.getFSDataset().onCompleteLazyPersist(bpId, blockId,
|
||||
creationTime, targetFiles, targetVolume);
|
||||
dataset.onCompleteLazyPersist(bpId, blockId,
|
||||
creationTime, targetFiles, targetVolume);
|
||||
succeeded = true;
|
||||
} catch (Exception e){
|
||||
FsDatasetImpl.LOG.warn(
|
||||
|
@ -244,7 +245,7 @@ class RamDiskAsyncLazyPersistService {
|
|||
+ bpId + "block Id: " + blockId, e);
|
||||
} finally {
|
||||
if (!succeeded) {
|
||||
datanode.getFSDataset().onFailLazyPersist(bpId, blockId);
|
||||
dataset.onFailLazyPersist(bpId, blockId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1253,7 +1253,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|||
|
||||
@Override
|
||||
public void onCompleteLazyPersist(String bpId, long blockId,
|
||||
long creationTime, File[] savedFiles, FsVolumeImpl targetVolume) {
|
||||
long creationTime, File[] savedFiles, FsVolumeSpi targetVolume) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,409 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.StorageType;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.datanode.*;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||
import org.apache.hadoop.util.DiskChecker;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
||||
|
||||
public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
|
||||
|
||||
private final DatanodeStorage storage = new DatanodeStorage(
|
||||
DatanodeStorage.generateUuid(), DatanodeStorage.State.NORMAL,
|
||||
StorageType.DEFAULT);
|
||||
|
||||
@Override
|
||||
public RollingLogs createRollingLogs(String bpid, String prefix)
|
||||
throws IOException {
|
||||
return new ExternalRollingLogs();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ExternalVolumeImpl> getVolumes() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addVolume(StorageLocation location, List<NamespaceInfo> nsInfos) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeVolumes(Collection<StorageLocation> volumes) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatanodeStorage getStorage(String storageUuid) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageReport[] getStorageReports(String bpid) throws IOException {
|
||||
StorageReport[] result = new StorageReport[1];
|
||||
result[0] = new StorageReport(storage, false, 0, 0, 0, 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExternalVolumeImpl getVolume(ExtendedBlock b) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> getVolumeInfoMap() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<FinalizedReplica> getFinalizedBlocks(String bpid) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<FinalizedReplica> getFinalizedBlocksOnPersistentStorage(String bpid) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkAndUpdate(String bpid, long blockId, File diskFile,
|
||||
File diskMetaFile, FsVolumeSpi vol) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
|
||||
throws IOException {
|
||||
return new LengthInputStream(null, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength(ExtendedBlock b) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public Replica getReplica(String bpid, long blockId) {
|
||||
return new ExternalReplica();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getReplicaString(String bpid, long blockId) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Block getStoredBlock(String bpid, long blkid) throws IOException {
|
||||
return new Block();
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset)
|
||||
throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkoff,
|
||||
long ckoff) throws IOException {
|
||||
return new ReplicaInputStreams(FileDescriptor.in, FileDescriptor.in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface createTemporary(StorageType t, ExtendedBlock b)
|
||||
throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface createRbw(StorageType t, ExtendedBlock b, boolean tf)
|
||||
throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface recoverRbw(ExtendedBlock b, long newGS,
|
||||
long minBytesRcvd, long maxBytesRcvd) throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface convertTemporaryToRbw(
|
||||
ExtendedBlock temporary) throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface append(ExtendedBlock b, long newGS,
|
||||
long expectedBlockLen) throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
|
||||
long expectedBlockLen) throws IOException {
|
||||
return new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
|
||||
throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalizeBlock(ExtendedBlock b) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unfinalizeBlock(ExtendedBlock b) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid) {
|
||||
final Map<DatanodeStorage, BlockListAsLongs> result =
|
||||
new HashMap<DatanodeStorage, BlockListAsLongs>();
|
||||
|
||||
result.put(storage, new BlockListAsLongs(null, null));
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Long> getCacheReport(String bpid) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(ExtendedBlock block) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkBlock(ExtendedBlock b, long minLength, ReplicaState state) throws ReplicaNotFoundException, UnexpectedReplicaStateException, FileNotFoundException, EOFException, IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValidBlock(ExtendedBlock b) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValidRbw(ExtendedBlock b) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invalidate(String bpid, Block[] invalidBlks) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cache(String bpid, long[] blockIds) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uncache(String bpid, long[] blockIds) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCached(String bpid, long blockId) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void checkDataDir() throws DiskErrorException {
|
||||
throw new DiskChecker.DiskErrorException(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void adjustCrcChannelPosition(ExtendedBlock b,
|
||||
ReplicaOutputStreams outs, int checksumSize) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasEnoughResource() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getReplicaVisibleLength(ExtendedBlock block) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
|
||||
throws IOException {
|
||||
return new ReplicaRecoveryInfo(0, 0, 0, ReplicaState.FINALIZED);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
|
||||
long recoveryId, long newBlockId, long newLength) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBlockPool(String bpid, Configuration conf) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdownBlockPool(String bpid) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteBlockPool(String bpid, boolean force) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b)
|
||||
throws IOException {
|
||||
return new BlockLocalPathInfo(null, "file", "metafile");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HdfsBlocksMetadata getHdfsBlocksMetadata(String bpid, long[] blockIds) throws IOException {
|
||||
return new HdfsBlocksMetadata(null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void enableTrash(String bpid) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void restoreTrash(String bpid) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean trashEnabled(String bpid) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setRollingUpgradeMarker(String bpid) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearRollingUpgradeMarker(String bpid) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void submitBackgroundSyncFileRangeRequest(ExtendedBlock block, FileDescriptor fd, long offset, long nbytes, int flags) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onCompleteLazyPersist(String bpId, long blockId, long creationTime, File[] savedFiles, ExternalVolumeImpl targetVolume) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailLazyPersist(String bpId, long blockId) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block, StorageType targetStorageType) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockPoolUsed(String bpid) throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDfsUsed() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCapacity() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRemaining() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStorageInfo() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumFailedVolumes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCacheUsed() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCacheCapacity() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumBlocksCached() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumBlocksFailedToCache() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumBlocksFailedToUncache() {
|
||||
return 0;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.datanode.Replica;
|
||||
|
||||
public class ExternalReplica implements Replica {
|
||||
|
||||
@Override
|
||||
public long getBlockId() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getGenerationStamp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaState getState() {
|
||||
return ReplicaState.FINALIZED;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumBytes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBytesOnDisk() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVisibleLength() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStorageUuid() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isOnTransientStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.datanode.ChunkChecksum;
|
||||
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
public class ExternalReplicaInPipeline implements ReplicaInPipelineInterface {
|
||||
|
||||
@Override
|
||||
public void setNumBytes(long bytesReceived) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBytesAcked() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytesAcked(long bytesAcked) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setLastChecksumAndDataLen(long dataLength, byte[] lastChecksum) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChunkChecksum getLastChecksumAndDataLen() {
|
||||
return new ChunkChecksum(0, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaOutputStreams createStreams(boolean isCreate,
|
||||
DataChecksum requestedChecksum) throws IOException {
|
||||
return new ReplicaOutputStreams(null, null, requestedChecksum, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockId() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getGenerationStamp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicaState getState() {
|
||||
return ReplicaState.FINALIZED;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumBytes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBytesOnDisk() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVisibleLength() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStorageUuid() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isOnTransientStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
|
||||
|
||||
public class ExternalRollingLogs implements RollingLogs {
|
||||
|
||||
private class ExternalLineIterator implements LineIterator {
|
||||
@Override
|
||||
public boolean isPrevious() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLastReadFromPrevious() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
private class ExternalAppender implements Appender {
|
||||
@Override
|
||||
public Appendable append(CharSequence cs) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appendable append(CharSequence cs, int i, int i1)
|
||||
throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appendable append(char c) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LineIterator iterator(boolean skipPrevious) throws IOException {
|
||||
return new ExternalLineIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Appender appender() {
|
||||
return new ExternalAppender();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean roll() throws IOException {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdfs.StorageType;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
|
||||
public class ExternalVolumeImpl implements FsVolumeSpi {
|
||||
|
||||
@Override
|
||||
public String[] getBlockPoolList() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvailable() throws IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getBasePath() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPath(String bpid) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getFinalizedDir(String bpid) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStorageID() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageType getStorageType() {
|
||||
return StorageType.DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTransientStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reserveSpaceForRbw(long bytesToReserve) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseReservedSpace(long bytesToRelease) {
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.extdataset;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.datanode.Replica;
|
||||
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Tests the ability to create external FsDatasetSpi implementations.
|
||||
*
|
||||
* The purpose of this suite of tests is to ensure that it is possible to
|
||||
* construct subclasses of FsDatasetSpi outside the Hadoop tree
|
||||
* (specifically, outside of the org.apache.hadoop.hdfs.server.datanode
|
||||
* package). This consists of creating subclasses of the two key classes
|
||||
* (FsDatasetSpi and FsVolumeSpi) *and* instances or subclasses of any
|
||||
* classes/interfaces their methods need to produce. If methods are added
|
||||
* to or changed in any superclasses, or if constructors of other classes
|
||||
* are changed, this package will fail to compile. In fixing this
|
||||
* compilation error, any new class dependencies should receive the same
|
||||
* treatment.
|
||||
*
|
||||
* It is worth noting what these tests do *not* accomplish. Just as
|
||||
* important as being able to produce instances of the appropriate classes
|
||||
* is being able to access all necessary methods on those classes as well
|
||||
* as on any additional classes accepted as inputs to FsDatasetSpi's
|
||||
* methods. It wouldn't be correct to mandate all methods be public, as
|
||||
* that would defeat encapsulation. Moreover, there is no natural
|
||||
* mechanism that would prevent a manually-constructed list of methods
|
||||
* from becoming stale. Rather than creating tests with no clear means of
|
||||
* maintaining them, this problem is left unsolved for now.
|
||||
*
|
||||
* Lastly, though merely compiling this package should signal success,
|
||||
* explicit testInstantiate* unit tests are included below so as to have a
|
||||
* tangible means of referring to each case.
|
||||
*/
|
||||
public class TestExternalDataset {
|
||||
|
||||
/**
|
||||
* Tests instantiating an FsDatasetSpi subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testInstantiateDatasetImpl() throws Throwable {
|
||||
FsDatasetSpi<?> inst = new ExternalDatasetImpl();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests instantiating a Replica subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testIntantiateExternalReplica() throws Throwable {
|
||||
Replica inst = new ExternalReplica();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests instantiating a ReplicaInPipelineInterface subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testInstantiateReplicaInPipeline() throws Throwable {
|
||||
ReplicaInPipelineInterface inst = new ExternalReplicaInPipeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests instantiating a RollingLogs subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testInstantiateRollingLogs() throws Throwable {
|
||||
RollingLogs inst = new ExternalRollingLogs();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests instantiating an FsVolumeSpi subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testInstantiateVolumeImpl() throws Throwable {
|
||||
FsVolumeSpi inst = new ExternalVolumeImpl();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue