001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hdfs.server.namenode;
019
020import org.apache.hadoop.HadoopIllegalArgumentException;
021import org.apache.hadoop.fs.InvalidPathException;
022import org.apache.hadoop.fs.PathIsNotDirectoryException;
023import org.apache.hadoop.fs.StorageType;
024import org.apache.hadoop.fs.UnresolvedLinkException;
025import org.apache.hadoop.fs.XAttr;
026import org.apache.hadoop.fs.XAttrSetFlag;
027import org.apache.hadoop.fs.permission.FsAction;
028import org.apache.hadoop.fs.permission.FsPermission;
029import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
030import org.apache.hadoop.hdfs.protocol.HdfsConstants;
031import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
032import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
033import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
034import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
035import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
036import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
037import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
038import org.apache.hadoop.hdfs.util.EnumCounters;
039import org.apache.hadoop.security.AccessControlException;
040
041import com.google.common.collect.Lists;
042
043import java.io.FileNotFoundException;
044import java.io.IOException;
045import java.util.Arrays;
046import java.util.EnumSet;
047import java.util.List;
048
049import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY;
050import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
051
052public class FSDirAttrOp {
053  static HdfsFileStatus setPermission(
054      FSDirectory fsd, final String src, FsPermission permission)
055      throws IOException {
056    if (FSDirectory.isExactReservedName(src)) {
057      throw new InvalidPathException(src);
058    }
059    FSPermissionChecker pc = fsd.getPermissionChecker();
060    INodesInPath iip;
061    fsd.writeLock();
062    try {
063      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
064      fsd.checkOwner(pc, iip);
065      unprotectedSetPermission(fsd, iip, permission);
066    } finally {
067      fsd.writeUnlock();
068    }
069    fsd.getEditLog().logSetPermissions(iip.getPath(), permission);
070    return fsd.getAuditFileInfo(iip);
071  }
072
073  static HdfsFileStatus setOwner(
074      FSDirectory fsd, String src, String username, String group)
075      throws IOException {
076    if (FSDirectory.isExactReservedName(src)) {
077      throw new InvalidPathException(src);
078    }
079    FSPermissionChecker pc = fsd.getPermissionChecker();
080    INodesInPath iip;
081    fsd.writeLock();
082    try {
083      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
084      fsd.checkOwner(pc, iip);
085      if (!pc.isSuperUser()) {
086        if (username != null && !pc.getUser().equals(username)) {
087          throw new AccessControlException("User " + username
088              + " is not a super user (non-super user cannot change owner).");
089        }
090        if (group != null && !pc.isMemberOfGroup(group)) {
091          throw new AccessControlException(
092              "User " + username + " does not belong to " + group);
093        }
094      }
095      unprotectedSetOwner(fsd, iip, username, group);
096    } finally {
097      fsd.writeUnlock();
098    }
099    fsd.getEditLog().logSetOwner(iip.getPath(), username, group);
100    return fsd.getAuditFileInfo(iip);
101  }
102
103  static HdfsFileStatus setTimes(
104      FSDirectory fsd, String src, long mtime, long atime)
105      throws IOException {
106    FSPermissionChecker pc = fsd.getPermissionChecker();
107
108    INodesInPath iip;
109    fsd.writeLock();
110    try {
111      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
112      // Write access is required to set access and modification times
113      if (fsd.isPermissionEnabled()) {
114        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
115      }
116      final INode inode = iip.getLastINode();
117      if (inode == null) {
118        throw new FileNotFoundException("File/Directory " + iip.getPath() +
119                                            " does not exist.");
120      }
121      boolean changed = unprotectedSetTimes(fsd, iip, mtime, atime, true);
122      if (changed) {
123        fsd.getEditLog().logTimes(iip.getPath(), mtime, atime);
124      }
125    } finally {
126      fsd.writeUnlock();
127    }
128    return fsd.getAuditFileInfo(iip);
129  }
130
131  static boolean setReplication(
132      FSDirectory fsd, BlockManager bm, String src, final short replication)
133      throws IOException {
134    bm.verifyReplication(src, replication, null);
135    final boolean isFile;
136    FSPermissionChecker pc = fsd.getPermissionChecker();
137    fsd.writeLock();
138    try {
139      final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.WRITE);
140      if (fsd.isPermissionEnabled()) {
141        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
142      }
143
144      final BlockInfo[] blocks = unprotectedSetReplication(fsd, iip,
145                                                           replication);
146      isFile = blocks != null;
147      if (isFile) {
148        fsd.getEditLog().logSetReplication(iip.getPath(), replication);
149      }
150    } finally {
151      fsd.writeUnlock();
152    }
153    return isFile;
154  }
155
156  static HdfsFileStatus unsetStoragePolicy(FSDirectory fsd, BlockManager bm,
157      String src) throws IOException {
158    return setStoragePolicy(fsd, bm, src,
159        HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, "unset");
160  }
161
162  static HdfsFileStatus setStoragePolicy(FSDirectory fsd, BlockManager bm,
163      String src, final String policyName) throws IOException {
164    // get the corresponding policy and make sure the policy name is valid
165    BlockStoragePolicy policy = bm.getStoragePolicy(policyName);
166    if (policy == null) {
167      throw new HadoopIllegalArgumentException(
168          "Cannot find a block policy with the name " + policyName);
169    }
170
171    return setStoragePolicy(fsd, bm, src, policy.getId(), "set");
172  }
173
174  static HdfsFileStatus setStoragePolicy(FSDirectory fsd, BlockManager bm,
175      String src, final byte policyId, final String operation)
176      throws IOException {
177    if (!fsd.isStoragePolicyEnabled()) {
178      throw new IOException(String.format(
179          "Failed to %s storage policy since %s is set to false.", operation,
180          DFS_STORAGE_POLICY_ENABLED_KEY));
181    }
182    FSPermissionChecker pc = fsd.getPermissionChecker();
183    INodesInPath iip;
184    fsd.writeLock();
185    try {
186      iip = fsd.resolvePath(pc, src, DirOp.WRITE);
187
188      if (fsd.isPermissionEnabled()) {
189        fsd.checkPathAccess(pc, iip, FsAction.WRITE);
190      }
191
192      unprotectedSetStoragePolicy(fsd, bm, iip, policyId);
193      fsd.getEditLog().logSetStoragePolicy(iip.getPath(), policyId);
194    } finally {
195      fsd.writeUnlock();
196    }
197    return fsd.getAuditFileInfo(iip);
198  }
199
200  static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)
201      throws IOException {
202    return bm.getStoragePolicies();
203  }
204
205  static BlockStoragePolicy getStoragePolicy(FSDirectory fsd, BlockManager bm,
206      String path) throws IOException {
207    FSPermissionChecker pc = fsd.getPermissionChecker();
208    fsd.readLock();
209    try {
210      final INodesInPath iip = fsd.resolvePath(pc, path, DirOp.READ_LINK);
211      if (fsd.isPermissionEnabled()) {
212        fsd.checkPathAccess(pc, iip, FsAction.READ);
213      }
214      INode inode = iip.getLastINode();
215      if (inode == null) {
216        throw new FileNotFoundException("File/Directory does not exist: "
217            + iip.getPath());
218      }
219      return bm.getStoragePolicy(inode.getStoragePolicyID());
220    } finally {
221      fsd.readUnlock();
222    }
223  }
224
225  static long getPreferredBlockSize(FSDirectory fsd, String src)
226      throws IOException {
227    FSPermissionChecker pc = fsd.getPermissionChecker();
228    fsd.readLock();
229    try {
230      final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.READ_LINK);
231      return INodeFile.valueOf(iip.getLastINode(), iip.getPath())
232          .getPreferredBlockSize();
233    } finally {
234      fsd.readUnlock();
235    }
236  }
237
238  /**
239   * Set the namespace, storagespace and typespace quota for a directory.
240   *
241   * Note: This does not support ".inodes" relative path.
242   */
243  static void setQuota(FSDirectory fsd, String src, long nsQuota, long ssQuota,
244      StorageType type) throws IOException {
245    FSPermissionChecker pc = fsd.getPermissionChecker();
246    if (fsd.isPermissionEnabled()) {
247      pc.checkSuperuserPrivilege();
248    }
249
250    fsd.writeLock();
251    try {
252      INodesInPath iip = fsd.resolvePath(pc, src, DirOp.WRITE);
253      INodeDirectory changed =
254          unprotectedSetQuota(fsd, iip, nsQuota, ssQuota, type);
255      if (changed != null) {
256        final QuotaCounts q = changed.getQuotaCounts();
257        if (type == null) {
258          fsd.getEditLog().logSetQuota(src, q.getNameSpace(), q.getStorageSpace());
259        } else {
260          fsd.getEditLog().logSetQuotaByStorageType(
261              src, q.getTypeSpaces().get(type), type);
262        }
263      }
264    } finally {
265      fsd.writeUnlock();
266    }
267  }
268
269  static void unprotectedSetPermission(
270      FSDirectory fsd, INodesInPath iip, FsPermission permissions)
271      throws FileNotFoundException, UnresolvedLinkException,
272             QuotaExceededException, SnapshotAccessControlException {
273    assert fsd.hasWriteLock();
274    final INode inode = FSDirectory.resolveLastINode(iip);
275    int snapshotId = iip.getLatestSnapshotId();
276    inode.setPermission(permissions, snapshotId);
277  }
278
279  static void unprotectedSetOwner(
280      FSDirectory fsd, INodesInPath iip, String username, String groupname)
281      throws FileNotFoundException, UnresolvedLinkException,
282      QuotaExceededException, SnapshotAccessControlException {
283    assert fsd.hasWriteLock();
284    final INode inode = FSDirectory.resolveLastINode(iip);
285    if (username != null) {
286      inode.setUser(username, iip.getLatestSnapshotId());
287    }
288    if (groupname != null) {
289      inode.setGroup(groupname, iip.getLatestSnapshotId());
290    }
291  }
292
293  static boolean setTimes(
294      FSDirectory fsd, INodesInPath iip, long mtime, long atime, boolean force)
295          throws QuotaExceededException {
296    fsd.writeLock();
297    try {
298      return unprotectedSetTimes(fsd, iip, mtime, atime, force);
299    } finally {
300      fsd.writeUnlock();
301    }
302  }
303
304  /**
305   * See {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String,
306   *     long, long, StorageType)}
307   * for the contract.
308   * Sets quota for for a directory.
309   * @return INodeDirectory if any of the quotas have changed. null otherwise.
310   * @throws FileNotFoundException if the path does not exist.
311   * @throws PathIsNotDirectoryException if the path is not a directory.
312   * @throws QuotaExceededException if the directory tree size is
313   *                                greater than the given quota
314   * @throws UnresolvedLinkException if a symlink is encountered in src.
315   * @throws SnapshotAccessControlException if path is in RO snapshot
316   */
317  static INodeDirectory unprotectedSetQuota(
318      FSDirectory fsd, INodesInPath iip, long nsQuota,
319      long ssQuota, StorageType type)
320      throws FileNotFoundException, PathIsNotDirectoryException,
321      QuotaExceededException, UnresolvedLinkException,
322      SnapshotAccessControlException, UnsupportedActionException {
323    assert fsd.hasWriteLock();
324    // sanity check
325    if ((nsQuota < 0 && nsQuota != HdfsConstants.QUOTA_DONT_SET &&
326         nsQuota != HdfsConstants.QUOTA_RESET) ||
327        (ssQuota < 0 && ssQuota != HdfsConstants.QUOTA_DONT_SET &&
328          ssQuota != HdfsConstants.QUOTA_RESET)) {
329      throw new IllegalArgumentException("Illegal value for nsQuota or " +
330                                         "ssQuota : " + nsQuota + " and " +
331                                         ssQuota);
332    }
333    // sanity check for quota by storage type
334    if ((type != null) && (!fsd.isQuotaByStorageTypeEnabled() ||
335        nsQuota != HdfsConstants.QUOTA_DONT_SET)) {
336      throw new UnsupportedActionException(
337          "Failed to set quota by storage type because either" +
338          DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY + " is set to " +
339          fsd.isQuotaByStorageTypeEnabled() + " or nsQuota value is illegal " +
340          nsQuota);
341    }
342
343    INodeDirectory dirNode =
344        INodeDirectory.valueOf(iip.getLastINode(), iip.getPath());
345    if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
346      throw new IllegalArgumentException("Cannot clear namespace quota on root.");
347    } else { // a directory inode
348      final QuotaCounts oldQuota = dirNode.getQuotaCounts();
349      final long oldNsQuota = oldQuota.getNameSpace();
350      final long oldSsQuota = oldQuota.getStorageSpace();
351
352      if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
353        nsQuota = oldNsQuota;
354      }
355      if (ssQuota == HdfsConstants.QUOTA_DONT_SET) {
356        ssQuota = oldSsQuota;
357      }
358
359      // unchanged space/namespace quota
360      if (type == null && oldNsQuota == nsQuota && oldSsQuota == ssQuota) {
361        return null;
362      }
363
364      // unchanged type quota
365      if (type != null) {
366          EnumCounters<StorageType> oldTypeQuotas = oldQuota.getTypeSpaces();
367          if (oldTypeQuotas != null && oldTypeQuotas.get(type) == ssQuota) {
368              return null;
369          }
370      }
371
372      final int latest = iip.getLatestSnapshotId();
373      dirNode.recordModification(latest);
374      dirNode.setQuota(fsd.getBlockStoragePolicySuite(), nsQuota, ssQuota, type);
375      return dirNode;
376    }
377  }
378
379  static BlockInfo[] unprotectedSetReplication(
380      FSDirectory fsd, INodesInPath iip, short replication)
381      throws QuotaExceededException, UnresolvedLinkException,
382             SnapshotAccessControlException {
383    assert fsd.hasWriteLock();
384
385    final BlockManager bm = fsd.getBlockManager();
386    final INode inode = iip.getLastINode();
387    if (inode == null || !inode.isFile()) {
388      return null;
389    }
390    INodeFile file = inode.asFile();
391
392    // Make sure the directory has sufficient quotas
393    short oldBR = file.getPreferredBlockReplication();
394
395    long size = file.computeFileSize(true, true);
396    // Ensure the quota does not exceed
397    if (oldBR < replication) {
398      fsd.updateCount(iip, 0L, size, oldBR, replication, true);
399    }
400
401    file.setFileReplication(replication, iip.getLatestSnapshotId());
402    short targetReplication = (short) Math.max(
403        replication, file.getPreferredBlockReplication());
404
405    if (oldBR > replication) {
406      fsd.updateCount(iip, 0L, size, oldBR, targetReplication, true);
407    }
408    for (BlockInfo b : file.getBlocks()) {
409      bm.setReplication(oldBR, targetReplication, b);
410    }
411
412    if (oldBR != -1) {
413      if (oldBR > targetReplication) {
414        FSDirectory.LOG.info("Decreasing replication from {} to {} for {}",
415                             oldBR, targetReplication, iip.getPath());
416      } else {
417        FSDirectory.LOG.info("Increasing replication from {} to {} for {}",
418                             oldBR, targetReplication, iip.getPath());
419      }
420    }
421    return file.getBlocks();
422  }
423
424  static void unprotectedSetStoragePolicy(FSDirectory fsd, BlockManager bm,
425      INodesInPath iip, final byte policyId)
426      throws IOException {
427    assert fsd.hasWriteLock();
428    final INode inode = iip.getLastINode();
429    if (inode == null) {
430      throw new FileNotFoundException("File/Directory does not exist: "
431          + iip.getPath());
432    }
433    final int snapshotId = iip.getLatestSnapshotId();
434    if (inode.isFile()) {
435      if (policyId != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
436        BlockStoragePolicy newPolicy = bm.getStoragePolicy(policyId);
437        if (newPolicy.isCopyOnCreateFile()) {
438          throw new HadoopIllegalArgumentException("Policy " + newPolicy
439              + " cannot be set after file creation.");
440        }
441      }
442
443      BlockStoragePolicy currentPolicy =
444          bm.getStoragePolicy(inode.getLocalStoragePolicyID());
445
446      if (currentPolicy != null && currentPolicy.isCopyOnCreateFile()) {
447        throw new HadoopIllegalArgumentException(
448            "Existing policy " + currentPolicy.getName() +
449                " cannot be changed after file creation.");
450      }
451      inode.asFile().setStoragePolicyID(policyId, snapshotId);
452    } else if (inode.isDirectory()) {
453      setDirStoragePolicy(fsd, iip, policyId);
454    } else {
455      throw new FileNotFoundException(iip.getPath()
456          + " is not a file or directory");
457    }
458  }
459
460  private static void setDirStoragePolicy(
461      FSDirectory fsd, INodesInPath iip, byte policyId) throws IOException {
462    INode inode = FSDirectory.resolveLastINode(iip);
463    List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
464    XAttr xAttr = BlockStoragePolicySuite.buildXAttr(policyId);
465    List<XAttr> newXAttrs = null;
466    if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
467      List<XAttr> toRemove = Lists.newArrayList();
468      toRemove.add(xAttr);
469      List<XAttr> removed = Lists.newArrayList();
470      newXAttrs = FSDirXAttrOp.filterINodeXAttrs(existingXAttrs, toRemove,
471          removed);
472    } else {
473      newXAttrs = FSDirXAttrOp.setINodeXAttrs(fsd, existingXAttrs,
474          Arrays.asList(xAttr),
475          EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
476    }
477    XAttrStorage.updateINodeXAttrs(inode, newXAttrs, iip.getLatestSnapshotId());
478  }
479
480  static boolean unprotectedSetTimes(
481      FSDirectory fsd, INodesInPath iip, long mtime, long atime, boolean force)
482          throws QuotaExceededException {
483    assert fsd.hasWriteLock();
484    boolean status = false;
485    INode inode = iip.getLastINode();
486    int latest = iip.getLatestSnapshotId();
487    if (mtime != -1) {
488      inode = inode.setModificationTime(mtime, latest);
489      status = true;
490    }
491
492    // if the last access time update was within the last precision interval,
493    // then no need to store access time
494    if (atime != -1 && (status || force
495        || atime > inode.getAccessTime() + fsd.getAccessTimePrecision())) {
496      inode.setAccessTime(atime, latest);
497      status = true;
498    }
499    return status;
500  }
501}