Skip to content
Merged
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 @@ -298,4 +298,8 @@ public DBCheckpoint getCheckpoint(boolean flush) {
public File getDbLocation() {
return dbLocation;
}

public CodecRegistry getCodecRegistry() {
return codecRegistry;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -528,6 +528,23 @@ public OzoneOutputStream createFile(String keyName, long size,
recursive);
}

/**
* List the status for a file or a directory and its contents.
*
* @param keyName Absolute path of the entry to be listed
* @param recursive For a directory if true all the descendants of a
* particular directory are listed
* @param startKey Key from which listing needs to start. If startKey exists
* its status is included in the final list.
* @param numEntries Number of entries to list from the start key
* @return list of file status
*/
public List<OzoneFileStatus> listStatus(String keyName, boolean recursive,
String startKey, long numEntries) throws IOException {
return proxy
.listStatus(volumeName, name, keyName, recursive, startKey, numEntries);
}

/**
* An Iterator to iterate over {@link OzoneKey} list.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
* 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
Expand Down Expand Up @@ -604,4 +604,21 @@ OzoneInputStream readFile(String volumeName, String bucketName,
OzoneOutputStream createFile(String volumeName, String bucketName,
String keyName, long size, ReplicationType type, ReplicationFactor factor,
boolean overWrite, boolean recursive) throws IOException;

/**
* List the status for a file or a directory and its contents.
*
* @param volumeName Volume name
* @param bucketName Bucket name
* @param keyName Absolute path of the entry to be listed
* @param recursive For a directory if true all the descendants of a
* particular directory are listed
* @param startKey Key from which listing needs to start. If startKey exists
* its status is included in the final list.
* @param numEntries Number of entries to list from the start key
* @return list of file status
*/
List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
String keyName, boolean recursive, String startKey, long numEntries)
throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -1113,4 +1113,12 @@ public OzoneOutputStream createFile(String volumeName, String bucketName,
throw new UnsupportedOperationException(
"Ozone REST protocol does not " + "support this operation.");
}

@Override
public List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
String keyName, boolean recursive, String startKey, long numEntries)
throws IOException {
throw new UnsupportedOperationException(
"Ozone REST protocol does not " + "support this operation.");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -993,6 +993,19 @@ public OzoneOutputStream createFile(String volumeName, String bucketName,
factor);
}

@Override
public List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
String keyName, boolean recursive, String startKey, long numEntries)
throws IOException {
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
.setBucketName(bucketName)
.setKeyName(keyName)
.build();
return ozoneManagerClient
.listStatus(keyArgs, recursive, startKey, numEntries);
}

private OzoneInputStream createInputStream(OmKeyInfo keyInfo,
String requestId) throws IOException {
LengthInputStream lengthInputStream = KeyInputStream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,7 @@ public static boolean isReadOnly(
case ListMultiPartUploadParts:
case GetFileStatus:
case LookupFile:
case ListStatus:
return true;
case CreateVolume:
case SetVolumeProperty:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,8 @@ public enum OMAction implements AuditAction {
GET_FILE_STATUS,
CREATE_DIRECTORY,
CREATE_FILE,
LOOKUP_FILE;
LOOKUP_FILE,
LIST_STATUS;

@Override
public String getAction() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.ozone.om.helpers;

import org.apache.hadoop.fs.Path;

import java.nio.file.Paths;

import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;

/**
* Utility class for OzoneFileSystem.
*/
public final class OzoneFSUtils {

private OzoneFSUtils() {}

/**
* Returns string representation of path after removing the leading slash.
*/
public static String pathToKey(Path path) {
return path.toString().substring(1);
}

/**
* Returns string representation of the input path parent. The function adds
* a trailing slash if it does not exist and returns an empty string if the
* parent is root.
*/
public static String getParent(String keyName) {
java.nio.file.Path parentDir = Paths.get(keyName).getParent();
if (parentDir == null) {
return "";
}
return addTrailingSlashIfNeeded(parentDir.toString());
}

/**
* The function returns immediate child of given ancestor in a particular
* descendant. For example if ancestor is /a/b and descendant is /a/b/c/d/e
* the function should return /a/b/c/. If the descendant itself is the
* immediate child then it is returned as is without adding a trailing slash.
* This is done to distinguish files from a directory as in ozone files do
* not carry a trailing slash.
*/
public static String getImmediateChild(String descendant, String ancestor) {
ancestor =
!ancestor.isEmpty() ? addTrailingSlashIfNeeded(ancestor) : ancestor;
if (!descendant.startsWith(ancestor)) {
return null;
}
java.nio.file.Path descendantPath = Paths.get(descendant);
java.nio.file.Path ancestorPath = Paths.get(ancestor);
int ancestorPathNameCount =
ancestor.isEmpty() ? 0 : ancestorPath.getNameCount();
if (descendantPath.getNameCount() - ancestorPathNameCount > 1) {
return addTrailingSlashIfNeeded(
ancestor + descendantPath.getName(ancestorPathNameCount));
}
return descendant;
}

public static String addTrailingSlashIfNeeded(String key) {
if (!key.endsWith(OZONE_URI_DELIMITER)) {
return key + OZONE_URI_DELIMITER;
} else {
return key;
}
}

public static boolean isFile(String keyName) {
return !keyName.endsWith(OZONE_URI_DELIMITER);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@

package org.apache.hadoop.ozone.om.helpers;

import org.apache.hadoop.fs.FSProtos.FileStatusProto;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.protocolPB.PBHelper;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;

import java.io.IOException;
import java.net.URI;
Expand Down Expand Up @@ -53,13 +53,14 @@ public OzoneFileStatus(String keyName) {
super(0, true, 0, 0, 0, getPath(keyName));
}

public FileStatusProto getProtobuf() throws IOException {
return PBHelper.convert(this);
public OzoneFileStatusProto getProtobuf() throws IOException {
return OzoneFileStatusProto.newBuilder().setStatus(PBHelper.convert(this))
.build();
}

public static OzoneFileStatus getFromProtobuf(FileStatusProto response)
public static OzoneFileStatus getFromProtobuf(OzoneFileStatusProto response)
throws IOException {
return new OzoneFileStatus(PBHelper.convert(response));
return new OzoneFileStatus(PBHelper.convert(response.getStatus()));
}

public static Path getPath(String keyName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -450,5 +450,19 @@ OpenKeySession createFile(OmKeyArgs keyArgs, boolean overWrite,
* invalid arguments
*/
OmKeyInfo lookupFile(OmKeyArgs keyArgs) throws IOException;

/**
* List the status for a file or a directory and its contents.
*
* @param keyArgs Key args
* @param recursive For a directory if true all the descendants of a
* particular directory are listed
* @param startKey Key from which listing needs to start. If startKey exists
* its status is included in the final list.
* @param numEntries Number of entries to list from the start key
* @return list of file status
*/
List<OzoneFileStatus> listStatus(OmKeyArgs keyArgs, boolean recursive,
String startKey, long numEntries) throws IOException;
}

Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,13 @@
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneFileStatusProto;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupFileRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupFileResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateFileRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateFileResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListStatusResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateDirectoryRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusRequest;
Expand Down Expand Up @@ -1281,14 +1287,13 @@ public OmKeyInfo lookupFile(OmKeyArgs args)
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.build();
OzoneManagerProtocolProtos.LookupFileRequest lookupFileRequest =
OzoneManagerProtocolProtos.LookupFileRequest.newBuilder()
LookupFileRequest lookupFileRequest = LookupFileRequest.newBuilder()
.setKeyArgs(keyArgs)
.build();
OMRequest omRequest = createOMRequest(Type.LookupFile)
.setLookupFileRequest(lookupFileRequest)
.build();
OzoneManagerProtocolProtos.LookupFileResponse resp =
LookupFileResponse resp =
handleError(submitRequest(omRequest)).getLookupFileResponse();
return OmKeyInfo.getFromProtobuf(resp.getKeyInfo());
}
Expand All @@ -1304,18 +1309,46 @@ public OpenKeySession createFile(OmKeyArgs args,
.setType(args.getType())
.setFactor(args.getFactor())
.build();
OzoneManagerProtocolProtos.CreateFileRequest createFileRequest =
OzoneManagerProtocolProtos.CreateFileRequest.newBuilder()
CreateFileRequest createFileRequest = CreateFileRequest.newBuilder()
.setKeyArgs(keyArgs)
.setIsOverwrite(overWrite)
.setIsRecursive(recursive)
.build();
OMRequest omRequest = createOMRequest(Type.CreateFile)
.setCreateFileRequest(createFileRequest)
.build();
OzoneManagerProtocolProtos.CreateFileResponse resp =
CreateFileResponse resp =
handleError(submitRequest(omRequest)).getCreateFileResponse();
return new OpenKeySession(resp.getID(),
OmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
}

@Override
public List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
String startKey, long numEntries) throws IOException {
KeyArgs keyArgs = KeyArgs.newBuilder()
.setVolumeName(args.getVolumeName())
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.build();
ListStatusRequest listStatusRequest =
ListStatusRequest.newBuilder()
.setKeyArgs(keyArgs)
.setRecursive(recursive)
.setStartKey(startKey)
.setNumEntries(numEntries)
.build();
OMRequest omRequest = createOMRequest(Type.ListStatus)
.setListStatusRequest(listStatusRequest)
.build();
ListStatusResponse listStatusResponse =
handleError(submitRequest(omRequest)).getListStatusResponse();
List<OzoneFileStatus> statusList =
new ArrayList<>(listStatusResponse.getStatusesCount());
for (OzoneFileStatusProto fileStatus : listStatusResponse
.getStatusesList()) {
statusList.add(OzoneFileStatus.getFromProtobuf(fileStatus));
}
return statusList;
}
}
20 changes: 19 additions & 1 deletion hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ enum Type {
CreateDirectory = 71;
CreateFile = 72;
LookupFile = 73;
ListStatus = 74;
}

message OMRequest {
Expand Down Expand Up @@ -141,6 +142,7 @@ message OMRequest {
optional CreateDirectoryRequest createDirectoryRequest = 71;
optional CreateFileRequest createFileRequest = 72;
optional LookupFileRequest lookupFileRequest = 73;
optional ListStatusRequest listStatusRequest = 74;
}

message OMResponse {
Expand Down Expand Up @@ -200,6 +202,7 @@ message OMResponse {
optional CreateDirectoryResponse createDirectoryResponse = 71;
optional CreateFileResponse createFileResponse = 72;
optional LookupFileResponse lookupFileResponse = 73;
optional ListStatusResponse listStatusResponse = 74;
}

enum Status {
Expand Down Expand Up @@ -561,12 +564,16 @@ message KeyInfo {
optional FileEncryptionInfoProto fileEncryptionInfo = 12;
}

message OzoneFileStatusProto {
required hadoop.fs.FileStatusProto status = 1;
}

message GetFileStatusRequest {
required KeyArgs keyArgs = 1;
}

message GetFileStatusResponse {
required hadoop.fs.FileStatusProto status = 1;
required OzoneFileStatusProto status = 1;
}

message CreateDirectoryRequest {
Expand Down Expand Up @@ -599,6 +606,17 @@ message LookupFileResponse {
optional KeyInfo keyInfo = 1;
}

message ListStatusRequest {
required KeyArgs keyArgs = 1;
required bool recursive = 2;
required string startKey = 3;
required uint64 numEntries = 4;
}

message ListStatusResponse {
repeated OzoneFileStatusProto statuses = 1;
}

message CreateKeyRequest {
required KeyArgs keyArgs = 1;
}
Expand Down
Loading