diff --git a/pkg/fs/fs.go b/pkg/fs/fs.go index 93ff8cd1d226..95ae19c08849 100644 --- a/pkg/fs/fs.go +++ b/pkg/fs/fs.go @@ -20,6 +20,7 @@ import ( "bytes" "context" "fmt" + "github.com/juicedata/juicefs/pkg/acl" "io" "os" "path" @@ -86,6 +87,9 @@ func (fs *FileStat) Mode() os.FileMode { if attr.Mode&01000 != 0 { mode |= os.ModeSticky } + if attr.AccessACL+attr.DefaultACL > 0 { + mode |= 1 << 18 + } switch attr.Typ { case meta.TypeDirectory: mode |= os.ModeDir @@ -627,6 +631,48 @@ func (fs *FileSystem) RemoveXattr(ctx meta.Context, p string, name string) (err return } +func (fs *FileSystem) GetFacl(ctx meta.Context, p string, acltype uint8, rule *acl.Rule) (err syscall.Errno) { + defer trace.StartRegion(context.TODO(), "fs.GetFacl").End() + l := vfs.NewLogContext(ctx) + defer func() { fs.log(l, "GetFacl (%s,%d): %s", p, acltype, errstr(err)) }() + fi, err := fs.resolve(ctx, p, true) + if err != 0 { + return + } + err = fs.m.GetFacl(ctx, fi.inode, acltype, rule) + return +} + +func (fs *FileSystem) SetFacl(ctx meta.Context, p string, acltype uint8, rule *acl.Rule) (err syscall.Errno) { + defer trace.StartRegion(context.TODO(), "fs.SetFacl").End() + l := vfs.NewLogContext(ctx) + defer func() { + fs.log(l, "SetFacl (%s,%d,%v): %s", p, acltype, rule, errstr(err)) + }() + fi, err := fs.resolve(ctx, p, true) + if err != 0 { + return + } + if acltype == acl.TypeDefault && fi.Mode().IsRegular() { + if rule.IsEmpty() { + return + } else { + return syscall.ENOTSUP + } + } + if rule.IsEmpty() { + oldRule := acl.EmptyRule() + if err = fs.m.GetFacl(ctx, fi.inode, acltype, oldRule); err != 0 { + return err + } + rule.Owner = oldRule.Owner + rule.Other = oldRule.Other + rule.Group = oldRule.Group & oldRule.Mask + } + err = fs.m.SetFacl(ctx, fi.inode, acltype, rule) + return +} + func (fs *FileSystem) lookup(ctx meta.Context, parent Ino, name string, inode *Ino, attr *Attr) (err syscall.Errno) { now := time.Now() if fs.conf.DirEntryTimeout > 0 || fs.conf.EntryTimeout > 0 { diff --git a/sdk/java/libjfs/main.go b/sdk/java/libjfs/main.go index adf971d09a3c..cdd045d6b845 100644 --- a/sdk/java/libjfs/main.go +++ b/sdk/java/libjfs/main.go @@ -31,6 +31,7 @@ import ( "bytes" "encoding/json" "fmt" + "github.com/juicedata/juicefs/pkg/acl" "io" "net/http" _ "net/http/pprof" @@ -882,6 +883,71 @@ func jfs_removeXattr(pid int, h int64, path *C.char, name *C.char) int { return errno(w.RemoveXattr(w.withPid(pid), C.GoString(path), C.GoString(name))) } +//export jfs_getfacl +func jfs_getfacl(pid int, h int64, path *C.char, acltype int, buf uintptr, blen int) int { + w := F(h) + if w == nil { + return EINVAL + } + rule := acl.EmptyRule() + err := w.GetFacl(w.withPid(pid), C.GoString(path), uint8(acltype), rule) + if err != 0 { + return errno(err) + } + wb := utils.NewNativeBuffer(toBuf(buf, blen)) + wb.Put16(rule.Owner) + wb.Put16(rule.Group) + wb.Put16(rule.Other) + wb.Put16(rule.Mask) + wb.Put16(uint16(len(rule.NamedUsers))) + wb.Put16(uint16(len(rule.NamedGroups))) + var off uintptr = 12 + for i, entry := range append(rule.NamedUsers, rule.NamedGroups...) { + var name string + if i < len(rule.NamedUsers) { + name = w.uid2name(entry.Id) + } else { + name = w.gid2name(entry.Id) + } + if wb.Left() < len(name)+1+2 { + return -100 + } + wb.Put([]byte(name)) + wb.Put8(0) + wb.Put16(entry.Perm) + } + return int(off) +} + +//export jfs_setfacl +func jfs_setfacl(pid int, h int64, path *C.char, acltype int, buf uintptr, alen int) int { + w := F(h) + if w == nil { + return EINVAL + } + rule := acl.EmptyRule() + r := utils.NewNativeBuffer(toBuf(buf, alen)) + rule.Owner = r.Get16() + rule.Group = r.Get16() + rule.Other = r.Get16() + rule.Mask = r.Get16() + namedusers := r.Get16() + namedgroups := r.Get16() + for i := uint16(0); i < namedusers+namedgroups; i++ { + name := string(r.Get(int(r.Get8()))) + var entry acl.Entry + entry.Perm = uint16(r.Get8()) + if i < namedusers { + entry.Id = w.lookupUid(name) + rule.NamedUsers = append(rule.NamedUsers, entry) + } else { + entry.Id = w.lookupGid(name) + rule.NamedGroups = append(rule.NamedGroups, entry) + } + } + return errno(w.SetFacl(w.withPid(pid), C.GoString(path), uint8(acltype), rule)) +} + //export jfs_readlink func jfs_readlink(pid int, h int64, link *C.char, buf uintptr, bufsize int) int { w := F(h) diff --git a/sdk/java/pom.xml b/sdk/java/pom.xml index 8359c228904a..31340d73c79a 100644 --- a/sdk/java/pom.xml +++ b/sdk/java/pom.xml @@ -42,6 +42,9 @@ ${argLine} false + + ${project.build.directory}/test-classes + @@ -217,6 +220,9 @@ conf + + src/test/resources + diff --git a/sdk/java/src/main/java/io/juicefs/JuiceFileSystemImpl.java b/sdk/java/src/main/java/io/juicefs/JuiceFileSystemImpl.java index 05d87230a52c..374cb8f79ae7 100644 --- a/sdk/java/src/main/java/io/juicefs/JuiceFileSystemImpl.java +++ b/sdk/java/src/main/java/io/juicefs/JuiceFileSystemImpl.java @@ -15,13 +15,11 @@ */ package io.juicefs; +import com.google.common.collect.Lists; import com.kenai.jffi.internal.StubLoader; import io.juicefs.exception.QuotaExceededException; import io.juicefs.metrics.JuiceFSInstrumentation; -import io.juicefs.utils.BgTaskUtil; -import io.juicefs.utils.ConsistentHash; -import io.juicefs.utils.NodesFetcher; -import io.juicefs.utils.NodesFetcherBuilder; +import io.juicefs.utils.*; import jnr.ffi.LibraryLoader; import jnr.ffi.Memory; import jnr.ffi.Pointer; @@ -35,8 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.*; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.*; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.security.AccessControlException; @@ -102,6 +99,8 @@ public class JuiceFileSystemImpl extends FileSystem { * hadoop compatibility */ private boolean withStreamCapability; + private Constructor fileStatusConstructor; + // constructor for BufferedFSOutputStreamWithStreamCapabilities private Constructor constructor; private Method setStorageIds; @@ -174,6 +173,10 @@ public static interface Libjfs { int jfs_removeXattr(long pid, long h, String path, String name); + int jfs_getfacl(long pid, long h, String path, int acltype, Pointer b, int len); + + int jfs_setfacl(long pid, long h, String path, int acltype, Pointer b, int len); + void jfs_set_callback(LogCallBack callBack); interface LogCallBack { @@ -431,6 +434,17 @@ public void initialize(URI uri, Configuration conf) throws IOException { throw new RuntimeException(e); } } + // for hadoop compatibility + boolean hasAclMtd = ReflectionUtil.hasMethod(FileStatus.class.getName(), "hasAcl", (String[]) null); + if (hasAclMtd) { + fileStatusConstructor = ReflectionUtil.getConstructor(FileStatus.class, + long.class, boolean.class, int.class, long.class, long.class, + long.class, FsPermission.class, String.class, String.class, Path.class, + Path.class, boolean.class, boolean.class, boolean.class); + if (fileStatusConstructor == null) { + throw new IOException("incompatible hadoop version"); + } + } uMask = FsPermission.getUMask(conf); String umaskStr = getConf(conf, "umask", null); @@ -1432,14 +1446,25 @@ private FileStatus newFileStatus(Path p, Pointer buf, int size, boolean readlink int mode = buf.getInt(0); boolean isdir = ((mode >>> 31) & 1) == 1; // Go int stickybit = (mode >>> 20) & 1; + boolean hasAcl = (mode >> 18 & 1) == 1; FsPermission perm = new FsPermission((short) ((mode & 0777) | (stickybit << 9))); + perm = new FsPermissionExtension(perm, hasAcl, false); long length = buf.getLongLong(4); long mtime = buf.getLongLong(12); long atime = buf.getLongLong(20); String user = buf.getString(28); String group = buf.getString(28 + user.length() + 1); assert (30 + user.length() + group.length() == size); - return new FileStatus(length, isdir, 1, blocksize, mtime, atime, perm, user, group, p); + + if (fileStatusConstructor == null) { + return new FileStatus(length, isdir, 1, blocksize, mtime, atime, perm, user, group, p); + } else { + try { + return fileStatusConstructor.newInstance(length, isdir, 1, blocksize, mtime, atime, perm, user, group, null, p, hasAcl, false, false); + } catch (Exception e) { + throw new IOException("construct fileStatus failed", e); + } + } } @Override @@ -1700,4 +1725,225 @@ public void removeXAttr(Path path, String name) throws IOException { if (r < 0) throw error(r, path); } + + @Override + public void modifyAclEntries(Path path, List aclSpec) throws IOException { + List existingEntries = getAllAclEntries(path); + List newAcl = AclTransformation.mergeAclEntries(existingEntries, aclSpec); + setAclInternal(path, newAcl); + } + + @Override + public void removeAclEntries(Path path, List aclSpec) throws IOException { + List existingEntries = getAllAclEntries(path); + List newAcl = AclTransformation.filterAclEntriesByAclSpec(existingEntries, aclSpec); + setAclInternal(path, newAcl); + } + + @Override + public void setAcl(Path path, List aclSpec) throws IOException { + List existingEntries = getAllAclEntries(path); + List newAcl = AclTransformation.replaceAclEntries(existingEntries, aclSpec); + setAclInternal(path, newAcl); + } + + private void setAclInternal(Path path, List aclSpec) throws IOException { + List aclEntries = AclTransformation.buildAndValidateAcl(Lists.newArrayList(aclSpec)); + ScopedAclEntries scoped = new ScopedAclEntries(aclEntries); + setAclInternal(path, AclEntryScope.ACCESS, scoped.getAccessEntries()); + setAclInternal(path, AclEntryScope.DEFAULT, scoped.getDefaultEntries()); + } + + private void removeAclInternal(Path path, AclEntryScope scope) throws IOException { + Pointer buf = Memory.allocate(Runtime.getRuntime(lib), 6 * 2); + buf.putShort(0, (short) -1); + buf.putShort(2, (short) -1); + buf.putShort(4, (short) -1); + buf.putShort(6, (short) -1); + buf.putShort(8, (short) 0); + buf.putShort(10, (short) 0); + int r = lib.jfs_setfacl(Thread.currentThread().getId(), handle, normalizePath(path), scope.ordinal() + 1, buf, + 6 * 2); + if (r == ENOATTR || r == ENODATA) + return; + if (r < 0) + throw error(r, path); + } + + @Override + public void removeDefaultAcl(Path path) throws IOException { + removeAclInternal(path, AclEntryScope.DEFAULT); + } + + @Override + public void removeAcl(Path path) throws IOException { + removeAclInternal(path, AclEntryScope.ACCESS); + removeAclInternal(path, AclEntryScope.DEFAULT); + } + + private void setAclInternal(Path path, AclEntryScope scope, List aclSpec) throws IOException { + if (aclSpec.size() == 0) + return; + short userperm = -1, groupperm = -1, otherperm = -1, maskperm = -1; + short namedusers = 0, namedgroups = 0; + int namedaclsize = 0; + for (AclEntry e : aclSpec) { + if (e.getName() != null) { + if (e.getType() == AclEntryType.USER) { + namedusers++; + } else { + namedgroups++; + } + namedaclsize += e.getName().getBytes("utf8").length + 2; + } else { + short perm = (short) e.getPermission().ordinal(); + switch (e.getType()) { + case USER: + userperm = perm; + break; + case GROUP: + groupperm = perm; + break; + case OTHER: + otherperm = perm; + break; + case MASK: + maskperm = perm; + break; + } + } + } + Pointer buf = Memory.allocate(Runtime.getRuntime(lib), 12 + namedaclsize); + buf.putShort(0, userperm); + buf.putShort(2, groupperm); + buf.putShort(4, otherperm); + buf.putShort(6, maskperm); + buf.putShort(8, namedusers); + buf.putShort(10, namedgroups); + int off = 12; + for (AclEntry e : aclSpec) { + String name = e.getName(); + if (name != null && e.getType() == AclEntryType.USER) { + byte[] nb = name.getBytes("utf8"); + buf.putByte(off, (byte) nb.length); + buf.put(off + 1, nb, 0, nb.length); + off += 1 + nb.length; + buf.putByte(off, (byte) e.getPermission().ordinal()); + off += 1; + } + } + for (AclEntry e : aclSpec) { + String name = e.getName(); + if (name != null && e.getType() == AclEntryType.GROUP) { + byte[] nb = name.getBytes("utf8"); + buf.putByte(off, (byte) nb.length); + buf.put(off + 1, nb, 0, nb.length); + off += 1 + nb.length; + buf.putByte(off, (byte) e.getPermission().ordinal()); + off += 1; + } + } + int r = lib.jfs_setfacl(Thread.currentThread().getId(), handle, normalizePath(path), scope.ordinal() + 1, buf, + 12 + namedaclsize); + if (r == ENOTSUP) { + throw new IOException("Invalid ACL: only directories may have a default ACL"); + } + if (r < 0) + throw error(r, path); + } + + private List getAclEntries(Path path, AclEntryScope scope) throws IOException { + int bufsize = 1024; + int r; + Pointer buf; + do { + bufsize *= 2; + buf = Memory.allocate(Runtime.getRuntime(lib), bufsize); + r = lib.jfs_getfacl(Thread.currentThread().getId(), handle, normalizePath(path), scope.ordinal() + 1, buf, + bufsize); + } while (r == -100); + if (r == ENOATTR || r == ENODATA) { + return Lists.newArrayList(); + } + if (r < 0) + throw error(r, path); + + int off = 0; + short userperm = buf.getShort(0); + short groupperm = buf.getShort(2); + short otherperm = buf.getShort(4); + short maskperm = buf.getShort(6); + short namedusers = buf.getShort(8); + short namedgroups = buf.getShort(10); + off += 12; + + List entries = new ArrayList<>(); + AclEntry.Builder builder = new AclEntry.Builder().setScope(scope); + if (userperm != -1) { + entries.add(builder.setType(AclEntryType.USER).setPermission(FsAction.values()[userperm]).build()); + } + if (groupperm != -1) { + entries.add(builder.setType(AclEntryType.GROUP).setPermission(FsAction.values()[groupperm]).build()); + } + if (otherperm != -1) { + entries.add(builder.setType(AclEntryType.OTHER).setPermission(FsAction.values()[otherperm]).build()); + } + if (maskperm != -1) { + entries.add(builder.setType(AclEntryType.MASK).setPermission(FsAction.values()[maskperm]).build()); + } + + for (int i = 0; i < namedusers + namedgroups; i++) { + String name = buf.getString(off); + off += name.length() + 1; + short perm = buf.getShort(off); + off += 2; + entries.add(builder.setType(i < namedusers ? AclEntryType.USER : AclEntryType.GROUP).setName(name) + .setPermission(FsAction.values()[perm]).build()); + } + Collections.sort(entries, AclTransformation.ACL_ENTRY_COMPARATOR); + return entries; + } + + /** + * include acl entries from permission + */ + private List getAllAclEntries(Path path) throws IOException { + List entries = getAclEntries(path, AclEntryScope.ACCESS); + if (entries.size() == 0) { + FsPermission perm = getFileStatus(path).getPermission(); + entries = AclUtil.getAclFromPermAndEntries(perm, entries); + } + entries.addAll(getAclEntries(path, AclEntryScope.DEFAULT)); + return entries; + } + + /** + * exclude acl entries from permission + */ + private List getAclEntries(Path path) throws IOException { + List res = new ArrayList<>(); + List accessEntries = getAclEntries(path, AclEntryScope.ACCESS); + // minimal 3 acls for ugo + if (accessEntries.size() != 0 && accessEntries.size() != 3) { + res.addAll(accessEntries.subList(1, accessEntries.size() - 2)); + } + res.addAll(getAclEntries(path, AclEntryScope.DEFAULT)); + return res; + } + + @Override + public AclStatus getAclStatus(Path path) throws IOException { + FileStatus st = getFileStatus(path); + List entries = getAclEntries(path); + AclStatus.Builder builder = new AclStatus.Builder().owner(st.getOwner()).group(st.getGroup()) + .stickyBit(st.getPermission().getStickyBit()).addEntries(entries); + try { + Class ab = AclStatus.Builder.class; + Method abm = ab.getDeclaredMethod("setPermission", FsPermission.class); + abm.setAccessible(true); + abm.invoke(builder, st.getPermission()); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException ignored) { + } + return builder.build(); + } } diff --git a/sdk/java/src/main/java/io/juicefs/utils/AclTransformation.java b/sdk/java/src/main/java/io/juicefs/utils/AclTransformation.java new file mode 100644 index 000000000000..f6b02fd7bfd9 --- /dev/null +++ b/sdk/java/src/main/java/io/juicefs/utils/AclTransformation.java @@ -0,0 +1,346 @@ +/** + * 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 io.juicefs.utils; + +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import org.apache.hadoop.fs.permission.*; + +import java.io.IOException; +import java.util.*; + +import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.*; + +/** + * AclTransformation defines the operations that can modify an ACL. All ACL + * modifications take as input an existing ACL and apply logic to add new + * entries, modify existing entries or remove old entries. Some operations also + * accept an ACL spec: a list of entries that further describes the requested + * change. Different operations interpret the ACL spec differently. In the + * case of adding an ACL to an inode that previously did not have one, the + * existing ACL can be a "minimal ACL" containing exactly 3 entries for owner, + * group and other, all derived from the {@link FsPermission} bits. + *

+ * The algorithms implemented here require sorted lists of ACL entries. For any + * existing ACL, it is assumed that the entries are sorted. This is because all + * ACL creation and modification is intended to go through these methods, and + * they all guarantee correct sort order in their outputs. However, an ACL spec + * is considered untrusted user input, so all operations pre-sort the ACL spec as + * the first step. + */ +public final class AclTransformation { + private static final int MAX_ENTRIES = 32; + + public static List filterAclEntriesByAclSpec(List existingAcl, List inAclSpec) throws AclException { + ValidatedAclSpec aclSpec = new ValidatedAclSpec(inAclSpec); + ArrayList aclBuilder = Lists.newArrayListWithCapacity(MAX_ENTRIES); + EnumMap providedMask = Maps.newEnumMap(AclEntryScope.class); + EnumSet maskDirty = EnumSet.noneOf(AclEntryScope.class); + EnumSet scopeDirty = EnumSet.noneOf(AclEntryScope.class); + for (AclEntry existingEntry : existingAcl) { + if (aclSpec.containsKey(existingEntry)) { + scopeDirty.add(existingEntry.getScope()); + if (existingEntry.getType() == MASK) { + maskDirty.add(existingEntry.getScope()); + } + } else { + if (existingEntry.getType() == MASK) { + providedMask.put(existingEntry.getScope(), existingEntry); + } else { + aclBuilder.add(existingEntry); + } + } + } + copyDefaultsIfNeeded(aclBuilder); + calculateMasks(aclBuilder, providedMask, maskDirty, scopeDirty); + return buildAndValidateAcl(aclBuilder); + } + + public static List mergeAclEntries(List existingAcl, List inAclSpec) throws AclException { + ValidatedAclSpec aclSpec = new ValidatedAclSpec(inAclSpec); + ArrayList aclBuilder = Lists.newArrayListWithCapacity(MAX_ENTRIES); + List foundAclSpecEntries = Lists.newArrayListWithCapacity(MAX_ENTRIES); + EnumMap providedMask = Maps.newEnumMap(AclEntryScope.class); + EnumSet maskDirty = EnumSet.noneOf(AclEntryScope.class); + EnumSet scopeDirty = EnumSet.noneOf(AclEntryScope.class); + for (AclEntry existingEntry : existingAcl) { + AclEntry aclSpecEntry = aclSpec.findByKey(existingEntry); + if (aclSpecEntry != null) { + foundAclSpecEntries.add(aclSpecEntry); + scopeDirty.add(aclSpecEntry.getScope()); + if (aclSpecEntry.getType() == MASK) { + providedMask.put(aclSpecEntry.getScope(), aclSpecEntry); + maskDirty.add(aclSpecEntry.getScope()); + } else { + aclBuilder.add(aclSpecEntry); + } + } else { + if (existingEntry.getType() == MASK) { + providedMask.put(existingEntry.getScope(), existingEntry); + } else { + aclBuilder.add(existingEntry); + } + } + } + // ACL spec entries that were not replacements are new additions. + for (AclEntry newEntry : aclSpec) { + if (Collections.binarySearch(foundAclSpecEntries, newEntry, ACL_ENTRY_COMPARATOR) < 0) { + scopeDirty.add(newEntry.getScope()); + if (newEntry.getType() == MASK) { + providedMask.put(newEntry.getScope(), newEntry); + maskDirty.add(newEntry.getScope()); + } else { + aclBuilder.add(newEntry); + } + } + } + copyDefaultsIfNeeded(aclBuilder); + calculateMasks(aclBuilder, providedMask, maskDirty, scopeDirty); + return buildAndValidateAcl(aclBuilder); + } + + public static List replaceAclEntries(List existingAcl, List inAclSpec) throws AclException { + ValidatedAclSpec aclSpec = new ValidatedAclSpec(inAclSpec); + ArrayList aclBuilder = Lists.newArrayListWithCapacity(MAX_ENTRIES); + // Replacement is done separately for each scope: access and default. + EnumMap providedMask = Maps.newEnumMap(AclEntryScope.class); + EnumSet maskDirty = EnumSet.noneOf(AclEntryScope.class); + EnumSet scopeDirty = EnumSet.noneOf(AclEntryScope.class); + for (AclEntry aclSpecEntry : aclSpec) { + scopeDirty.add(aclSpecEntry.getScope()); + if (aclSpecEntry.getType() == MASK) { + providedMask.put(aclSpecEntry.getScope(), aclSpecEntry); + maskDirty.add(aclSpecEntry.getScope()); + } else { + aclBuilder.add(aclSpecEntry); + } + } + // Copy existing entries if the scope was not replaced. + for (AclEntry existingEntry : existingAcl) { + if (!scopeDirty.contains(existingEntry.getScope())) { + if (existingEntry.getType() == MASK) { + providedMask.put(existingEntry.getScope(), existingEntry); + } else { + aclBuilder.add(existingEntry); + } + } + } + copyDefaultsIfNeeded(aclBuilder); + calculateMasks(aclBuilder, providedMask, maskDirty, scopeDirty); + return buildAndValidateAcl(aclBuilder); + } + + private AclTransformation() { + } + + public static final Comparator ACL_ENTRY_COMPARATOR = new Comparator() { + @Override + public int compare(AclEntry entry1, AclEntry entry2) { + return ComparisonChain.start().compare(entry1.getScope(), entry2.getScope(), Ordering.explicit(ACCESS, DEFAULT)).compare(entry1.getType(), entry2.getType(), Ordering.explicit(USER, GROUP, MASK, OTHER)).compare(entry1.getName(), entry2.getName(), Ordering.natural().nullsFirst()).result(); + } + }; + + public static List buildAndValidateAcl(ArrayList aclBuilder) throws AclException { + aclBuilder.trimToSize(); + Collections.sort(aclBuilder, ACL_ENTRY_COMPARATOR); + // Full iteration to check for duplicates and invalid named entries. + AclEntry prevEntry = null; + for (AclEntry entry : aclBuilder) { + if (prevEntry != null && ACL_ENTRY_COMPARATOR.compare(prevEntry, entry) == 0) { + throw new AclException("Invalid ACL: multiple entries with same scope, type and name."); + } + if (entry.getName() != null && (entry.getType() == MASK || entry.getType() == OTHER)) { + throw new AclException("Invalid ACL: this entry type must not have a name: " + entry + "."); + } + prevEntry = entry; + } + + ScopedAclEntries scopedEntries = new ScopedAclEntries(aclBuilder); + checkMaxEntries(scopedEntries); + + // Search for the required base access entries. If there is a default ACL, + // then do the same check on the default entries. + for (AclEntryType type : EnumSet.of(USER, GROUP, OTHER)) { + AclEntry accessEntryKey = new AclEntry.Builder().setScope(ACCESS).setType(type).build(); + if (Collections.binarySearch(scopedEntries.getAccessEntries(), accessEntryKey, ACL_ENTRY_COMPARATOR) < 0) { + throw new AclException("Invalid ACL: the user, group and other entries are required."); + } + if (!scopedEntries.getDefaultEntries().isEmpty()) { + AclEntry defaultEntryKey = new AclEntry.Builder().setScope(DEFAULT).setType(type).build(); + if (Collections.binarySearch(scopedEntries.getDefaultEntries(), defaultEntryKey, ACL_ENTRY_COMPARATOR) < 0) { + throw new AclException("Invalid default ACL: the user, group and other entries are required."); + } + } + } + return Collections.unmodifiableList(aclBuilder); + } + + private static void checkMaxEntries(ScopedAclEntries scopedEntries) throws AclException { + List accessEntries = scopedEntries.getAccessEntries(); + List defaultEntries = scopedEntries.getDefaultEntries(); + if (accessEntries.size() > MAX_ENTRIES) { + throw new AclException("Invalid ACL: ACL has " + accessEntries.size() + " access entries, which exceeds maximum of " + MAX_ENTRIES + "."); + } + if (defaultEntries.size() > MAX_ENTRIES) { + throw new AclException("Invalid ACL: ACL has " + defaultEntries.size() + " default entries, which exceeds maximum of " + MAX_ENTRIES + "."); + } + } + + private static void calculateMasks(List aclBuilder, EnumMap providedMask, EnumSet maskDirty, EnumSet scopeDirty) throws AclException { + EnumSet scopeFound = EnumSet.noneOf(AclEntryScope.class); + EnumMap unionPerms = Maps.newEnumMap(AclEntryScope.class); + EnumSet maskNeeded = EnumSet.noneOf(AclEntryScope.class); + // Determine which scopes are present, which scopes need a mask, and the + // union of group class permissions in each scope. + for (AclEntry entry : aclBuilder) { + scopeFound.add(entry.getScope()); + if (entry.getType() == GROUP || entry.getName() != null) { + FsAction scopeUnionPerms = unionPerms.get(entry.getScope()); + if (scopeUnionPerms == null) { + scopeUnionPerms = FsAction.NONE; + } + unionPerms.put(entry.getScope(), scopeUnionPerms.or(entry.getPermission())); + } + if (entry.getName() != null) { + maskNeeded.add(entry.getScope()); + } + } + // Add mask entry if needed in each scope. + for (AclEntryScope scope : scopeFound) { + if (!providedMask.containsKey(scope) && maskNeeded.contains(scope) && maskDirty.contains(scope)) { + // Caller explicitly removed mask entry, but it's required. + throw new AclException("Invalid ACL: mask is required and cannot be deleted."); + } else if (providedMask.containsKey(scope) && (!scopeDirty.contains(scope) || maskDirty.contains(scope))) { + // Caller explicitly provided new mask, or we are preserving the existing + // mask in an unchanged scope. + aclBuilder.add(providedMask.get(scope)); + } else if (maskNeeded.contains(scope) || providedMask.containsKey(scope)) { + // Otherwise, if there are maskable entries present, or the ACL + // previously had a mask, then recalculate a mask automatically. + aclBuilder.add(new AclEntry.Builder().setScope(scope).setType(MASK).setPermission(unionPerms.get(scope)).build()); + } + } + } + + private static void copyDefaultsIfNeeded(List aclBuilder) { + Collections.sort(aclBuilder, ACL_ENTRY_COMPARATOR); + ScopedAclEntries scopedEntries = new ScopedAclEntries(aclBuilder); + if (!scopedEntries.getDefaultEntries().isEmpty()) { + List accessEntries = scopedEntries.getAccessEntries(); + List defaultEntries = scopedEntries.getDefaultEntries(); + List copiedEntries = Lists.newArrayListWithCapacity(3); + for (AclEntryType type : EnumSet.of(USER, GROUP, OTHER)) { + AclEntry defaultEntryKey = new AclEntry.Builder().setScope(DEFAULT).setType(type).build(); + int defaultEntryIndex = Collections.binarySearch(defaultEntries, defaultEntryKey, ACL_ENTRY_COMPARATOR); + if (defaultEntryIndex < 0) { + AclEntry accessEntryKey = new AclEntry.Builder().setScope(ACCESS).setType(type).build(); + int accessEntryIndex = Collections.binarySearch(accessEntries, accessEntryKey, ACL_ENTRY_COMPARATOR); + if (accessEntryIndex >= 0) { + copiedEntries.add(new AclEntry.Builder().setScope(DEFAULT).setType(type).setPermission(accessEntries.get(accessEntryIndex).getPermission()).build()); + } + } + } + // Add all copied entries when done to prevent potential issues with binary + // search on a modified aclBulider during the main loop. + aclBuilder.addAll(copiedEntries); + } + } + + private static final class ValidatedAclSpec implements Iterable { + private final List aclSpec; + + /** + * Creates a ValidatedAclSpec by pre-validating and sorting the given ACL + * entries. Pre-validation checks that it does not exceed the maximum + * entries. This check is performed before modifying the ACL, and it's + * actually insufficient for enforcing the maximum number of entries. + * Transformation logic can create additional entries automatically,such as + * the mask and some of the default entries, so we also need additional + * checks during transformation. The up-front check is still valuable here + * so that we don't run a lot of expensive transformation logic while + * holding the namesystem lock for an attacker who intentionally sent a huge + * ACL spec. + * + * @param aclSpec List containing unvalidated input ACL spec + * @throws AclException if validation fails + */ + public ValidatedAclSpec(List aclSpec) throws AclException { + Collections.sort(aclSpec, ACL_ENTRY_COMPARATOR); + checkMaxEntries(new ScopedAclEntries(aclSpec)); + this.aclSpec = aclSpec; + } + + /** + * Returns true if this contains an entry matching the given key. An ACL + * entry's key consists of scope, type and name (but not permission). + * + * @param key AclEntry search key + * @return boolean true if found + */ + public boolean containsKey(AclEntry key) { + return Collections.binarySearch(aclSpec, key, ACL_ENTRY_COMPARATOR) >= 0; + } + + /** + * Returns the entry matching the given key or null if not found. An ACL + * entry's key consists of scope, type and name (but not permission). + * + * @param key AclEntry search key + * @return AclEntry entry matching the given key or null if not found + */ + public AclEntry findByKey(AclEntry key) { + int index = Collections.binarySearch(aclSpec, key, ACL_ENTRY_COMPARATOR); + if (index >= 0) { + return aclSpec.get(index); + } + return null; + } + + @Override + public Iterator iterator() { + return aclSpec.iterator(); + } + } + + public static class AclException extends IOException { + private static final long serialVersionUID = 1L; + + /** + * Creates a new AclException. + * + * @param message String message + */ + public AclException(String message) { + super(message); + } + + /** + * Creates a new AclException. + * + * @param message String message + * @param cause The cause of the exception + */ + public AclException(String message, Throwable cause) { + super(message, cause); + } + } +} diff --git a/sdk/java/src/main/java/io/juicefs/utils/FsPermissionExtension.java b/sdk/java/src/main/java/io/juicefs/utils/FsPermissionExtension.java new file mode 100644 index 000000000000..c0dee7e0f0df --- /dev/null +++ b/sdk/java/src/main/java/io/juicefs/utils/FsPermissionExtension.java @@ -0,0 +1,63 @@ +/** + * 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 io.juicefs.utils; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.permission.FsPermission; + +/** + * HDFS permission subclass used to indicate an ACL is present and/or that the + * underlying file/dir is encrypted. The ACL/encrypted bits are not visible + * directly to users of {@link FsPermission} serialization. This is + * done for backwards compatibility in case any existing clients assume the + * value of FsPermission is in a particular range. + */ +@InterfaceAudience.Private +public class FsPermissionExtension extends FsPermission { + private final static short ACL_BIT = 1 << 12; + private final static short ENCRYPTED_BIT = 1 << 13; + private final boolean aclBit; + private final boolean encryptedBit; + + /** + * Constructs a new FsPermissionExtension based on the given FsPermission. + * + * @param perm FsPermission containing permission bits + */ + public FsPermissionExtension(FsPermission perm, boolean hasAcl, + boolean isEncrypted) { + super(perm.toShort()); + aclBit = hasAcl; + encryptedBit = isEncrypted; + } + + @Override + public short toExtendedShort() { + return (short) (toShort() | + (aclBit ? ACL_BIT : 0) | (encryptedBit ? ENCRYPTED_BIT : 0)); + } + + public boolean getAclBit() { + return aclBit; + } + + @Override + public boolean getEncryptedBit() { + return encryptedBit; + } +} \ No newline at end of file diff --git a/sdk/java/src/main/java/io/juicefs/utils/ReflectionUtil.java b/sdk/java/src/main/java/io/juicefs/utils/ReflectionUtil.java new file mode 100644 index 000000000000..418be1736022 --- /dev/null +++ b/sdk/java/src/main/java/io/juicefs/utils/ReflectionUtil.java @@ -0,0 +1,53 @@ +/* + * JuiceFS, Copyright 2024 Juicedata, Inc. + * + * Licensed 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 io.juicefs.utils; + +import java.lang.reflect.Constructor; + +public class ReflectionUtil { + public static boolean hasMethod(String className, String method, String[] params) { + try { + Class[] classes = null; + if (params != null) { + classes = new Class[params.length]; + for (int i = 0; i < params.length; i++) { + classes[i] = Class.forName(params[i], false, Thread.currentThread().getContextClassLoader()); + } + } + return hasMethod(className, method, classes); + } catch (ClassNotFoundException e) { + return false; + } + } + + public static boolean hasMethod(String className, String method, Class[] params) { + try { + Class clazz = Class.forName(className, false, Thread.currentThread().getContextClassLoader()); + clazz.getDeclaredMethod(method, params); + } catch (ClassNotFoundException | NoSuchMethodException e) { + return false; + } + return true; + } + + public static Constructor getConstructor(Class clazz, Class... params) { + try { + return clazz.getConstructor(params); + } catch (NoSuchMethodException e) { + return null; + } + } +} diff --git a/sdk/java/src/test/java/io/juicefs/JuiceFileSystemTest.java b/sdk/java/src/test/java/io/juicefs/JuiceFileSystemTest.java index 102a817805a5..c9ad0891f9b3 100644 --- a/sdk/java/src/test/java/io/juicefs/JuiceFileSystemTest.java +++ b/sdk/java/src/test/java/io/juicefs/JuiceFileSystemTest.java @@ -16,13 +16,14 @@ package io.juicefs; +import com.google.common.collect.Lists; +import io.juicefs.utils.AclTransformation; import junit.framework.TestCase; import org.apache.commons.io.IOUtils; import org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.*; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; @@ -43,6 +44,11 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_CHECKPOINT_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; +import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.*; +import static org.apache.hadoop.fs.permission.FsAction.*; +import static org.apache.hadoop.fs.permission.FsAction.ALL; import static org.junit.Assert.assertArrayEquals; public class JuiceFileSystemTest extends TestCase { @@ -944,4 +950,174 @@ public void testConcurrentCreate() throws Exception { pool.shutdown(); pool.awaitTermination(1, TimeUnit.MINUTES); } + + private boolean tryAccess(Path path, String user, String[] group, FsAction action) throws Exception { + UserGroupInformation testUser = UserGroupInformation.createUserForTesting(user, group); + FileSystem fs = testUser.doAs((PrivilegedExceptionAction) () -> { + Configuration conf = new Configuration(); + conf.set("juicefs.grouping", ""); + return FileSystem.get(conf); + }); + + boolean canAccess; + try { + fs.access(path, action); + canAccess = true; + } catch (AccessControlException e) { + canAccess = false; + } + return canAccess; + } + static AclEntry aclEntry(AclEntryScope scope, AclEntryType type, FsAction permission) { + return new AclEntry.Builder().setScope(scope).setType(type).setPermission(permission).build(); + } + + static AclEntry aclEntry(AclEntryScope scope, AclEntryType type, String name, FsAction permission) { + return new AclEntry.Builder().setScope(scope).setType(type).setName(name).setPermission(permission).build(); + } + + public void testAcl() throws Exception { + List acls = Lists.newArrayList( + aclEntry(DEFAULT, USER, "foo", ALL) + ); + Path p = new Path("/testacldir"); + fs.delete(p, true); + fs.mkdirs(p); + fs.setAcl(p, acls); + Path childFile = new Path(p, "file"); + fs.create(childFile).close(); + assertTrue(tryAccess(childFile, "foo", new String[]{"nogrp"}, WRITE)); + assertFalse(tryAccess(childFile, "wrong", new String[]{"nogrp"}, WRITE)); + assertEquals(fs.getFileStatus(childFile).getPermission().getGroupAction(), READ_WRITE); + + Path childDir = new Path(p, "dir"); + fs.mkdirs(childDir); + assertEquals(fs.getFileStatus(childDir).getPermission().getGroupAction(), ALL); + } + + public void testAclException() throws Exception { + List acls = Lists.newArrayList( + aclEntry(ACCESS, USER, "foo", ALL) + ); + Path p = new Path("/test_acl_exception"); + fs.delete(p, true); + fs.mkdirs(p); + try { + fs.setAcl(p, acls); + fail("Invalid ACL: the user, group and other entries are required."); + } catch (AclTransformation.AclException ignored) { + } + } + + public void testDefaultAclExistingDirFile() throws Exception { + Path parent = new Path("/testDefaultAclExistingDirFile"); + fs.delete(parent, true); + fs.mkdirs(parent); + // the old acls + List acls1 = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", ALL)); + // the new acls + List acls2 = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_EXECUTE)); + // set parent to old acl + fs.setAcl(parent, acls1); + + Path childDir = new Path(parent, "childDir"); + fs.mkdirs(childDir); + // the sub directory should also have the old acl + AclEntry[] childDirExpectedAcl = new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE), aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", ALL), aclEntry(DEFAULT, GROUP, READ_EXECUTE), + aclEntry(DEFAULT, MASK, ALL), aclEntry(DEFAULT, OTHER, READ_EXECUTE) }; + AclStatus childDirAcl = fs.getAclStatus(childDir); + assertArrayEquals(childDirExpectedAcl, childDirAcl.getEntries().toArray()); + + Path childFile = new Path(childDir, "childFile"); + // the sub file should also have the old acl + fs.create(childFile).close(); + AclEntry[] childFileExpectedAcl = new AclEntry[] { aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, GROUP, READ_EXECUTE) }; + AclStatus childFileAcl = fs.getAclStatus(childFile); + assertArrayEquals(childFileExpectedAcl, childFileAcl.getEntries().toArray()); + + // now change parent to new acls + fs.setAcl(parent, acls2); + + // sub directory and sub file should still have the old acls + childDirAcl = fs.getAclStatus(childDir); + assertArrayEquals(childDirExpectedAcl, childDirAcl.getEntries().toArray()); + childFileAcl = fs.getAclStatus(childFile); + assertArrayEquals(childFileExpectedAcl, childFileAcl.getEntries().toArray()); + + // now remove the parent acls + fs.removeAcl(parent); + + // sub directory and sub file should still have the old acls + childDirAcl = fs.getAclStatus(childDir); + assertArrayEquals(childDirExpectedAcl, childDirAcl.getEntries().toArray()); + childFileAcl = fs.getAclStatus(childFile); + assertArrayEquals(childFileExpectedAcl, childFileAcl.getEntries().toArray()); + + // check changing the access mode of the file + // mask out the access of group other for testing + fs.setPermission(childFile, new FsPermission((short) 0640)); + boolean canAccess = tryAccess(childFile, "other", new String[] { "other" }, READ); + assertFalse(canAccess); + fs.delete(parent, true); + } + + public void testAccessAclNotInherited() throws IOException { + Path parent = new Path("/testAccessAclNotInherited"); + fs.delete(parent, true); + fs.mkdirs(parent); + // parent have both access acl and default acl + List acls = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, USER, ALL), aclEntry(ACCESS, GROUP, READ), aclEntry(ACCESS, OTHER, READ), + aclEntry(ACCESS, USER, "bar", ALL)); + fs.setAcl(parent, acls); + AclEntry[] expectedAcl = new AclEntry[] { aclEntry(ACCESS, USER, "bar", ALL), aclEntry(ACCESS, GROUP, READ), + aclEntry(DEFAULT, USER, ALL), aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), + aclEntry(DEFAULT, GROUP, READ), aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, READ) }; + AclStatus dirAcl = fs.getAclStatus(parent); + assertArrayEquals(expectedAcl, dirAcl.getEntries().toArray()); + + Path childDir = new Path(parent, "childDir"); + fs.mkdirs(childDir); + // subdirectory should only have the default acl inherited + AclEntry[] childDirExpectedAcl = new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, GROUP, READ), aclEntry(DEFAULT, USER, ALL), + aclEntry(DEFAULT, USER, "foo", READ_EXECUTE), aclEntry(DEFAULT, GROUP, READ), + aclEntry(DEFAULT, MASK, READ_EXECUTE), aclEntry(DEFAULT, OTHER, READ) }; + AclStatus childDirAcl = fs.getAclStatus(childDir); + assertArrayEquals(childDirExpectedAcl, childDirAcl.getEntries().toArray()); + + Path childFile = new Path(parent, "childFile"); + fs.create(childFile).close(); + // sub file should only have the default acl inherited + AclEntry[] childFileExpectedAcl = new AclEntry[] { aclEntry(ACCESS, USER, "foo", READ_EXECUTE), + aclEntry(ACCESS, GROUP, READ) }; + AclStatus childFileAcl = fs.getAclStatus(childFile); + assertArrayEquals(childFileExpectedAcl, childFileAcl.getEntries().toArray()); + + fs.delete(parent, true); + } + + public void testFileStatusWithAcl() throws Exception { + List acls = Lists.newArrayList( + aclEntry(ACCESS, USER, ALL), + aclEntry(ACCESS, USER, "foo", ALL), + aclEntry(ACCESS, OTHER, ALL), + aclEntry(ACCESS, GROUP, ALL) + ); + Path p = new Path("/test_acl_status"); + fs.delete(p, true); + fs.mkdirs(p); + FileStatus pStatus = fs.getFileStatus(p); + assertFalse(pStatus.hasAcl()); + + Path f = new Path(p, "f"); + fs.create(f).close(); + fs.setAcl(f, acls); + FileStatus[] fileStatuses = fs.listStatus(p); + assertTrue(fileStatuses[0].getPermission().getAclBit()); + assertTrue(fileStatuses[0].hasAcl()); + } } diff --git a/sdk/java/src/test/java/io/juicefs/acl/TestAclCLI.java b/sdk/java/src/test/java/io/juicefs/acl/TestAclCLI.java new file mode 100644 index 000000000000..e03999d5efff --- /dev/null +++ b/sdk/java/src/test/java/io/juicefs/acl/TestAclCLI.java @@ -0,0 +1,79 @@ +/** + * 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 io.juicefs.acl; + +import org.apache.hadoop.cli.CLITestHelperDFS; +import org.apache.hadoop.cli.util.CLICommand; +import org.apache.hadoop.cli.util.CommandExecutor.Result; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestAclCLI extends CLITestHelperDFS { + private String vol = null; + private String username = null; + + protected void initConf() { + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true); + conf.setBoolean( + DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY, false); + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + initConf(); + vol = "jfs://dev/"; + username = System.getProperty("user.name"); + } + + @After + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + @Override + protected String getTestFile() { + return "testAclCLI.xml"; + } + + @Override + protected String expandCommand(final String cmd) { + String expCmd = cmd; + expCmd = expCmd.replaceAll("NAMENODE", vol); + expCmd = expCmd.replaceAll("USERNAME", username); + expCmd = expCmd.replaceAll("#LF#", + System.getProperty("line.separator")); + expCmd = super.expandCommand(expCmd); + return expCmd; + } + + @Override + protected Result execute(CLICommand cmd) throws Exception { + return cmd.getExecutor(vol, conf).executeCommand(cmd.getCmd()); + } + + @Test + @Override + public void testAll() { + super.testAll(); + } +} diff --git a/sdk/java/src/test/resources/testAclCLI.xml b/sdk/java/src/test/resources/testAclCLI.xml new file mode 100644 index 000000000000..120da68e4f69 --- /dev/null +++ b/sdk/java/src/test/resources/testAclCLI.xml @@ -0,0 +1,1075 @@ + + + + + + + + test + + + + + + getfacl: basic permissions + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -getfacl /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + # file: /file1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + group::r-- + + + SubstringComparator + other::r-- + + + + + getfacl: basic permissions for directory + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + group::r-x + + + SubstringComparator + other::r-x + + + + + setfacl : Add an ACL + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -setfacl -m user:bob:r-- /file1 + -fs NAMENODE -getfacl /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + # file: /file1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + user:bob:r-- + + + SubstringComparator + group::r-- + + + SubstringComparator + mask::r-- + + + SubstringComparator + other::r-- + + + + + setfacl : Add multiple ACLs at once + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /file1 + -fs NAMENODE -getfacl /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + # file: /file1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + user:bob:r-- + + + SubstringComparator + group::r-- + + + SubstringComparator + group:users:r-x + + + SubstringComparator + mask::r-x + + + SubstringComparator + other::r-- + + + + + setfacl : Remove an ACL + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -setfacl -m user:bob:r--,user:charlie:r-x /file1 + -fs NAMENODE -setfacl -x user:bob /file1 + -fs NAMENODE -getfacl /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + # file: /file1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + user:charlie:r-x + + + SubstringComparator + group::r-- + + + SubstringComparator + other::r-- + + + RegexpAcrossOutputComparator + .*(?!bob)* + + + + + setfacl : Add default ACL + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + user:bob:r-- + + + SubstringComparator + group::r-x + + + SubstringComparator + group:users:r-x + + + SubstringComparator + mask::r-x + + + SubstringComparator + other::r-x + + + SubstringComparator + default:user::rwx + + + SubstringComparator + default:user:charlie:r-x + + + SubstringComparator + default:group::r-x + + + SubstringComparator + default:group:admin:rwx + + + SubstringComparator + default:mask::rwx + + + SubstringComparator + default:other::r-x + + + + + setfacl : Add minimal default ACL + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m default:user::rwx /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + group::r-x + + + SubstringComparator + other::r-x + + + SubstringComparator + default:user::rwx + + + SubstringComparator + default:group::r-x + + + SubstringComparator + default:other::r-x + + + RegexpAcrossOutputComparator + .*(?!default\:mask)* + + + + + setfacl : try adding default ACL to file + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + setfacl: Invalid ACL: only directories may have a default ACL + + + + + setfacl : Remove one default ACL + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -setfacl -x default:user:charlie /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + user:bob:r-- + + + SubstringComparator + group::r-x + + + SubstringComparator + group:users:r-x + + + SubstringComparator + mask::r-x + + + SubstringComparator + other::r-x + + + SubstringComparator + default:user::rwx + + + SubstringComparator + default:group::r-x + + + SubstringComparator + default:group:admin:rwx + + + SubstringComparator + default:mask::rwx + + + SubstringComparator + default:other::r-x + + + RegexpAcrossOutputComparator + .*(?!default:user:charlie).* + + + + + setfacl : Remove all default ACL + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:bob:r--,group:users:r-x /dir1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -setfacl -k /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + user:bob:r-- + + + SubstringComparator + group::r-x + + + SubstringComparator + group:users:r-x + + + SubstringComparator + mask::r-x + + + SubstringComparator + other::r-x + + + RegexpAcrossOutputComparator + .*(?!default).* + + + + + setfacl : Remove all but base ACLs for a directory + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -setfacl -b /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rwx + + + SubstringComparator + group::r-x + + + SubstringComparator + other::r-x + + + RegexpAcrossOutputComparator + .*(?!charlie).* + + + RegexpAcrossOutputComparator + .*(?!default).* + + + RegexpAcrossOutputComparator + .*(?!admin).* + + + + + setfacl : Remove all but base ACLs for a file + + -fs NAMENODE -touchz /file1 + -fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /file1 + -fs NAMENODE -setfacl -b /file1 + -fs NAMENODE -getfacl /file1 + + + -fs NAMENODE -rm /file1 + + + + SubstringComparator + # file: /file1 + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + group::r-- + + + SubstringComparator + other::r-- + + + RegexpAcrossOutputComparator + .*(?!charlie).* + + + RegexpAcrossOutputComparator + .*(?!admin).* + + + + + setfacl : check inherit default ACL to file + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -touchz /dir1/file + -fs NAMENODE -getfacl /dir1/file + + + -fs NAMENODE -rm -R /dir1 + + + + SubstringComparator + # file: /dir1/file + + + SubstringComparator + # owner: USERNAME + + + SubstringComparator + # group: supergroup + + + SubstringComparator + user::rw- + + + SubstringComparator + user:charlie:r-x + + + SubstringComparator + group::r-x + + + SubstringComparator + group:admin:rwx + + + SubstringComparator + mask::rw- + + + SubstringComparator + other::r-- + + + RegexpAcrossOutputComparator + .*(?!default).* + + + + + setfacl : check inherit default ACL to dir + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m default:user:charlie:r-x,default:group:admin:rwx /dir1 + -fs NAMENODE -mkdir /dir1/dir2 + -fs NAMENODE -getfacl /dir1/dir2 + + + -fs NAMENODE -rm -R /dir1 + + + + ExactLineComparator + # file: /dir1/dir2 + + + ExactLineComparator + # owner: USERNAME + + + ExactLineComparator + # group: supergroup + + + ExactLineComparator + user::rwx + + + ExactLineComparator + user:charlie:r-x + + + ExactLineComparator + group::r-x + + + RegexpComparator + ^group:admin:rwx\b.* + + + ExactLineComparator + mask::rwx + + + ExactLineComparator + default:user::rwx + + + ExactLineComparator + default:user:charlie:r-x + + + ExactLineComparator + default:group::r-x + + + ExactLineComparator + default:group:admin:rwx + + + ExactLineComparator + default:mask::rwx + + + ExactLineComparator + default:other::r-x + + + ExactLineComparator + other::r-x + + + + + getfacl -R : recursive + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1 + -fs NAMENODE -mkdir /dir1/dir2 + -fs NAMENODE -setfacl -m user:user1:r-x,group:users:rwx /dir1/dir2 + -fs NAMENODE -getfacl -R /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + ExactComparator + # file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:user1:r-x#LF#group::r-x#LF#group:users:rwx#LF#mask::rwx#LF#other::r-x#LF##LF# + + + + + setfacl -R : recursive + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -mkdir /dir1/dir2 + -fs NAMENODE -setfacl -R -m user:charlie:r-x,group:admin:rwx /dir1 + -fs NAMENODE -getfacl -R /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + ExactComparator + # file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:r-x#LF#group::r-x#LF#group:admin:rwx#LF#mask::rwx#LF#other::r-x#LF##LF# + + + + + setfacl --set : Set full set of ACLs + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1 + -fs NAMENODE -setfacl --set user::rw-,group::r--,other::r--,user:user1:r-x,group:users:rw- /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + ExactComparator + # file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:user1:r-x#LF#group::r--#LF#group:users:rw-#LF#mask::rwx#LF#other::r--#LF##LF# + + + + + setfacl -x mask : remove mask entry along with other ACL entries + + -fs NAMENODE -mkdir /dir1 + -fs NAMENODE -setfacl -m user:charlie:r-x,group:admin:rwx /dir1 + -fs NAMENODE -setfacl -x mask::,user:charlie,group:admin /dir1 + -fs NAMENODE -getfacl /dir1 + + + -fs NAMENODE -rm -R /dir1 + + + + ExactComparator + # file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#group::r-x#LF#other::r-x#LF##LF# + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +