Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,9 @@ protected ObjectStore() {
proxy = null;
}

@VisibleForTesting
/**
* Returns the ClientProtocol of the ObjectStore.
*/
public ClientProtocol getClientProxy() {
return proxy;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public TestRootedOzoneFileSystem(boolean setDefaultFs) {
private static FileSystem fs;
private static RootedOzoneFileSystem ofs;
private static ObjectStore objectStore;
private static BasicRootedOzoneClientAdapterImpl adapter;
private static BasicRootedOzoneClientAdapterImpl impl;
private static Trash trash;

private static String volumeName;
Expand Down Expand Up @@ -147,7 +147,7 @@ public static void init() throws Exception {
fs = FileSystem.get(conf);
trash = new Trash(conf);
ofs = (RootedOzoneFileSystem) fs;
adapter = (BasicRootedOzoneClientAdapterImpl) ofs.getAdapter();
impl = ofs.getImpl();
}

@AfterClass
Expand Down Expand Up @@ -664,7 +664,7 @@ private void listStatusRecursiveHelper(Path curPath, List<FileStatus> result)
*/
private List<FileStatus> callAdapterListStatus(String pathStr,
boolean recursive, String startPath, long numEntries) throws IOException {
return adapter.listStatus(pathStr, recursive, startPath, numEntries,
return impl.listStatus(pathStr, recursive, startPath, numEntries,
ofs.getUri(), ofs.getWorkingDirectory(), ofs.getUsername())
.stream().map(ofs::convertFileStatus).collect(Collectors.toList());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -228,11 +228,6 @@ public void renameKey(String key, String newKeyName) throws IOException {
bucket.renameKey(key, newKeyName);
}

@Override
public void rename(String pathStr, String newPath) throws IOException {
throw new IOException("Please use renameKey instead for o3fs.");
}

/**
* Helper method to create an directory specified by key name in bucket.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,13 +88,11 @@
* <p>
* For full featured version use RootedOzoneClientAdapterImpl.
*/
public class BasicRootedOzoneClientAdapterImpl
implements OzoneClientAdapter {
public class BasicRootedOzoneClientAdapterImpl {

static final Logger LOG =
LoggerFactory.getLogger(BasicRootedOzoneClientAdapterImpl.class);

private OzoneClient ozoneClient;
private ObjectStore objectStore;
private ClientProtocol proxy;
private ReplicationType replicationType;
Expand Down Expand Up @@ -173,17 +171,15 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort,
int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);

OzoneClient ozoneClient;
if (OmUtils.isOmHAServiceId(conf, omHost)) {
// omHost is listed as one of the service ids in the config,
// thus we should treat omHost as omServiceId
this.ozoneClient =
OzoneClientFactory.getRpcClient(omHost, conf);
ozoneClient = OzoneClientFactory.getRpcClient(omHost, conf);
} else if (StringUtils.isNotEmpty(omHost) && omPort != -1) {
this.ozoneClient =
OzoneClientFactory.getRpcClient(omHost, omPort, conf);
ozoneClient = OzoneClientFactory.getRpcClient(omHost, omPort, conf);
} else {
this.ozoneClient =
OzoneClientFactory.getRpcClient(conf);
ozoneClient = OzoneClientFactory.getRpcClient(conf);
}
objectStore = ozoneClient.getObjectStore();
proxy = objectStore.getClientProxy();
Expand Down Expand Up @@ -240,7 +236,7 @@ private OzoneBucket getBucket(String volumeStr, String bucketStr,
if (getVolEx.getResult().equals(VOLUME_NOT_FOUND)) {
// Volume doesn't exist. Create it
try {
objectStore.createVolume(volumeStr);
proxy.createVolume(volumeStr);
} catch (OMException newVolEx) {
// Ignore the case where another client created the volume
if (!newVolEx.getResult().equals(VOLUME_ALREADY_EXISTS)) {
Expand Down Expand Up @@ -273,17 +269,14 @@ private OzoneBucket getBucket(String volumeStr, String bucketStr,
return bucket;
}

@Override
public short getDefaultReplication() {
return (short) replicationFactor.getValue();
}

@Override
public void close() throws IOException {
ozoneClient.close();
proxy.close();
}

@Override
public InputStream readFile(String pathStr) throws IOException {
incrementCounter(Statistic.OBJECTS_READ, 1);
OFSPath ofsPath = new OFSPath(pathStr);
Expand All @@ -306,7 +299,6 @@ protected void incrementCounter(Statistic objectsRead, long count) {
//noop: Use RootedOzoneClientAdapterImpl which supports statistics.
}

@Override
public OzoneFSOutputStream createFile(String pathStr, short replication,
boolean overWrite, boolean recursive) throws IOException {
incrementCounter(Statistic.OBJECTS_CREATED, 1);
Expand Down Expand Up @@ -341,11 +333,6 @@ public OzoneFSOutputStream createFile(String pathStr, short replication,
}
}

@Override
public void renameKey(String key, String newKeyName) throws IOException {
throw new IOException("OFS doesn't support renameKey, use rename instead.");
}

/**
* Rename a path into another.
*
Expand All @@ -357,7 +344,6 @@ public void renameKey(String key, String newKeyName) throws IOException {
* @param newPath Target path
* @throws IOException
*/
@Override
public void rename(String path, String newPath) throws IOException {
incrementCounter(Statistic.OBJECTS_RENAMED, 1);
OFSPath ofsPath = new OFSPath(path);
Expand Down Expand Up @@ -400,7 +386,6 @@ void rename(OzoneBucket bucket, String path, String newPath)
* @param pathStr path to be created as directory
* @return true if the key is created, false otherwise
*/
@Override
public boolean createDirectory(String pathStr) throws IOException {
LOG.trace("creating dir for path: {}", pathStr);
incrementCounter(Statistic.OBJECTS_CREATED, 1);
Expand All @@ -411,7 +396,7 @@ public boolean createDirectory(String pathStr) throws IOException {
}
if (ofsPath.getBucketName().isEmpty()) {
// Create volume only
objectStore.createVolume(ofsPath.getVolumeName());
proxy.createVolume(ofsPath.getVolumeName());
return true;
}
String keyStr = ofsPath.getKeyName();
Expand Down Expand Up @@ -439,7 +424,6 @@ public boolean createDirectory(String pathStr) throws IOException {
* @param path path to a key to be deleted
* @return true if the key is deleted, false otherwise
*/
@Override
public boolean deleteObject(String path) {
LOG.trace("issuing delete for path to key: {}", path);
incrementCounter(Statistic.OBJECTS_DELETED, 1);
Expand Down Expand Up @@ -485,7 +469,6 @@ private boolean areInSameBucket(List<String> keyNameList) {
* @param keyNameList key name list to be deleted
* @return true if the key deletion is successful, false otherwise
*/
@Override
public boolean deleteObjects(List<String> keyNameList) {
if (keyNameList.size() == 0) {
return true;
Expand Down Expand Up @@ -539,7 +522,7 @@ public FileStatusAdapter getFileStatus(String path, URI uri,
return getFileStatusAdapterForRoot(uri);
}
if (ofsPath.isVolume()) {
OzoneVolume volume = objectStore.getVolume(ofsPath.getVolumeName());
OzoneVolume volume = proxy.getVolumeDetails(ofsPath.getVolumeName());
return getFileStatusAdapterForVolume(volume, uri);
}
try {
Expand Down Expand Up @@ -614,7 +597,6 @@ public Collection<FileStatus> getTrashRoots(boolean allUsers,
return ret;
}

@Override
public Iterator<BasicKeyInfo> listKeys(String pathStr) {
incrementCounter(Statistic.OBJECTS_LIST, 1);
OFSPath ofsPath = new OFSPath(pathStr);
Expand All @@ -637,12 +619,11 @@ private List<FileStatusAdapter> listStatusRoot(
URI uri, Path workingDir, String username) throws IOException {

OFSPath ofsStartPath = new OFSPath(startPath);
// list volumes
Iterator<? extends OzoneVolume> iter = objectStore.listVolumesByUser(
username, null, ofsStartPath.getVolumeName());
List<FileStatusAdapter> res = new ArrayList<>();
while (iter.hasNext() && res.size() < numEntries) {
OzoneVolume volume = iter.next();
// list volumes
List<OzoneVolume> listVolumes = proxy.listVolumes(
username, null, ofsStartPath.getVolumeName(), (int)numEntries);
for (OzoneVolume volume : listVolumes) {
res.add(getFileStatusAdapterForVolume(volume, uri));
if (recursive) {
String pathStrNextVolume = volume.getName();
Expand All @@ -662,7 +643,7 @@ private List<FileStatusAdapter> listStatusVolume(String volumeStr,

OFSPath ofsStartPath = new OFSPath(startPath);
// list buckets in the volume
OzoneVolume volume = objectStore.getVolume(volumeStr);
OzoneVolume volume = proxy.getVolumeDetails(volumeStr);
Iterator<? extends OzoneBucket> iter =
volume.listBuckets(null, ofsStartPath.getBucketName());
List<FileStatusAdapter> res = new ArrayList<>();
Expand Down Expand Up @@ -756,36 +737,36 @@ public List<FileStatusAdapter> listStatus(String pathStr, boolean recursive,
}
}

@Override
public Token<OzoneTokenIdentifier> getDelegationToken(String renewer)
throws IOException {
if (!securityEnabled) {
return null;
}
Token<OzoneTokenIdentifier> token = ozoneClient.getObjectStore()
.getDelegationToken(renewer == null ? null : new Text(renewer));
Token<OzoneTokenIdentifier> token = proxy.getDelegationToken(
renewer == null ? null : new Text(renewer));
token.setKind(OzoneTokenIdentifier.KIND_NAME);
return token;

}

public ObjectStore getObjectStore() {
return objectStore;
}

@Override
public KeyProvider getKeyProvider() throws IOException {
return objectStore.getKeyProvider();
return proxy.getKeyProvider();
}

@Override
public URI getKeyProviderUri() throws IOException {
return objectStore.getKeyProviderUri();
return proxy.getKeyProviderUri();
}

@Override
public String getCanonicalServiceName() {
return objectStore.getCanonicalServiceName();
return proxy.getCanonicalServiceName();
}

public OzoneVolume getVolumeDetails(String volumeName) throws IOException {
return proxy.getVolumeDetails(volumeName);
}

public void deleteVolume(String volumeName) throws IOException {
proxy.deleteVolume(volumeName);
}

/**
Expand Down
Loading