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 java.io.IOException;
021import java.util.EnumSet;
022import java.util.List;
023import java.util.NavigableMap;
024import java.util.TreeMap;
025
026import com.google.common.annotations.VisibleForTesting;
027import com.google.common.base.Preconditions;
028import com.google.common.collect.Lists;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.crypto.CipherSuite;
031import org.apache.hadoop.crypto.CryptoProtocolVersion;
032import org.apache.hadoop.fs.UnresolvedLinkException;
033import org.apache.hadoop.fs.XAttr;
034import org.apache.hadoop.fs.XAttrSetFlag;
035import org.apache.hadoop.hdfs.DFSConfigKeys;
036import org.apache.hadoop.hdfs.XAttrHelper;
037import org.apache.hadoop.hdfs.protocol.EncryptionZone;
038import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
039import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
040import org.apache.hadoop.hdfs.protocolPB.PBHelper;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044
045import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
046import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants
047    .CRYPTO_XATTR_ENCRYPTION_ZONE;
048
049/**
050 * Manages the list of encryption zones in the filesystem.
051 * <p/>
052 * The EncryptionZoneManager has its own lock, but relies on the FSDirectory
053 * lock being held for many operations. The FSDirectory lock should not be
054 * taken if the manager lock is already held.
055 */
056public class EncryptionZoneManager {
057
058  public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
059      .class);
060
061  @VisibleForTesting
062  private boolean allowNestedEZ = false;
063
064  /**
065   * EncryptionZoneInt is the internal representation of an encryption zone. The
066   * external representation of an EZ is embodied in an EncryptionZone and
067   * contains the EZ's pathname.
068   */
069  private static class EncryptionZoneInt {
070    private final long inodeId;
071    private final CipherSuite suite;
072    private final CryptoProtocolVersion version;
073    private final String keyName;
074
075    EncryptionZoneInt(long inodeId, CipherSuite suite,
076        CryptoProtocolVersion version, String keyName) {
077      Preconditions.checkArgument(suite != CipherSuite.UNKNOWN);
078      Preconditions.checkArgument(version != CryptoProtocolVersion.UNKNOWN);
079      this.inodeId = inodeId;
080      this.suite = suite;
081      this.version = version;
082      this.keyName = keyName;
083    }
084
085    long getINodeId() {
086      return inodeId;
087    }
088
089    CipherSuite getSuite() {
090      return suite;
091    }
092
093    CryptoProtocolVersion getVersion() { return version; }
094
095    String getKeyName() {
096      return keyName;
097    }
098  }
099
100  private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
101  private final FSDirectory dir;
102  private final int maxListEncryptionZonesResponses;
103
104  /**
105   * Construct a new EncryptionZoneManager.
106   *
107   * @param dir Enclosing FSDirectory
108   */
109  public EncryptionZoneManager(FSDirectory dir, Configuration conf) {
110    this.dir = dir;
111    maxListEncryptionZonesResponses = conf.getInt(
112        DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
113        DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT
114    );
115    Preconditions.checkArgument(maxListEncryptionZonesResponses >= 0,
116        DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES + " " +
117            "must be a positive integer."
118    );
119  }
120
121  /**
122   * Add a new encryption zone.
123   * <p/>
124   * Called while holding the FSDirectory lock.
125   *
126   * @param inodeId of the encryption zone
127   * @param keyName encryption zone key name
128   */
129  void addEncryptionZone(Long inodeId, CipherSuite suite,
130      CryptoProtocolVersion version, String keyName) {
131    assert dir.hasWriteLock();
132    unprotectedAddEncryptionZone(inodeId, suite, version, keyName);
133  }
134
135  /**
136   * Add a new encryption zone.
137   * <p/>
138   * Does not assume that the FSDirectory lock is held.
139   *
140   * @param inodeId of the encryption zone
141   * @param keyName encryption zone key name
142   */
143  void unprotectedAddEncryptionZone(Long inodeId,
144      CipherSuite suite, CryptoProtocolVersion version, String keyName) {
145    final EncryptionZoneInt ez = new EncryptionZoneInt(
146        inodeId, suite, version, keyName);
147    if (encryptionZones == null) {
148      encryptionZones = new TreeMap<Long, EncryptionZoneInt>();
149    }
150    encryptionZones.put(inodeId, ez);
151  }
152
153  /**
154   * Remove an encryption zone.
155   * <p/>
156   * Called while holding the FSDirectory lock.
157   */
158  void removeEncryptionZone(Long inodeId) {
159    assert dir.hasWriteLock();
160    if (hasCreatedEncryptionZone()) {
161      encryptionZones.remove(inodeId);
162    }
163  }
164
165  /**
166   * Returns true if an IIP is within an encryption zone.
167   * <p/>
168   * Called while holding the FSDirectory lock.
169   */
170  boolean isInAnEZ(INodesInPath iip)
171      throws UnresolvedLinkException, SnapshotAccessControlException {
172    assert dir.hasReadLock();
173    return (getEncryptionZoneForPath(iip) != null);
174  }
175
176  /**
177   * Returns the path of the EncryptionZoneInt.
178   * <p/>
179   * Called while holding the FSDirectory lock.
180   */
181  private String getFullPathName(EncryptionZoneInt ezi) {
182    assert dir.hasReadLock();
183    return dir.getInode(ezi.getINodeId()).getFullPathName();
184  }
185
186  /**
187   * Get the key name for an encryption zone. Returns null if <tt>iip</tt> is
188   * not within an encryption zone.
189   * <p/>
190   * Called while holding the FSDirectory lock.
191   */
192  String getKeyName(final INodesInPath iip) {
193    assert dir.hasReadLock();
194    EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
195    if (ezi == null) {
196      return null;
197    }
198    return ezi.getKeyName();
199  }
200
201  /**
202   * Looks up the EncryptionZoneInt for a path within an encryption zone.
203   * Returns null if path is not within an EZ.
204   * <p/>
205   * Must be called while holding the manager lock.
206   */
207  private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
208    assert dir.hasReadLock();
209    Preconditions.checkNotNull(iip);
210    if (!hasCreatedEncryptionZone()) {
211      return null;
212    }
213    final INode[] inodes = iip.getINodes();
214    for (int i = inodes.length - 1; i >= 0; i--) {
215      final INode inode = inodes[i];
216      if (inode != null) {
217        final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
218        if (ezi != null) {
219          return ezi;
220        }
221      }
222    }
223    return null;
224  }
225
226  /**
227   * Returns an EncryptionZone representing the ez for a given path.
228   * Returns an empty marker EncryptionZone if path is not in an ez.
229   *
230   * @param iip The INodesInPath of the path to check
231   * @return the EncryptionZone representing the ez for the path.
232   */
233  EncryptionZone getEZINodeForPath(INodesInPath iip) {
234    final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
235    if (ezi == null) {
236      return null;
237    } else {
238      return new EncryptionZone(ezi.getINodeId(), getFullPathName(ezi),
239          ezi.getSuite(), ezi.getVersion(), ezi.getKeyName());
240    }
241  }
242
243  /**
244   * Throws an exception if the provided path cannot be renamed into the
245   * destination because of differing encryption zones.
246   * <p/>
247   * Called while holding the FSDirectory lock.
248   *
249   * @param srcIIP source IIP
250   * @param dstIIP destination IIP
251   * @param src    source path, used for debugging
252   * @throws IOException if the src cannot be renamed to the dst
253   */
254  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
255      throws IOException {
256    assert dir.hasReadLock();
257    final EncryptionZoneInt srcEZI = getEncryptionZoneForPath(srcIIP);
258    final EncryptionZoneInt dstEZI = getEncryptionZoneForPath(dstIIP);
259    final boolean srcInEZ = (srcEZI != null);
260    final boolean dstInEZ = (dstEZI != null);
261    if (srcInEZ) {
262      if (!dstInEZ) {
263        if (srcEZI.getINodeId() == srcIIP.getLastINode().getId()) {
264          // src is ez root and dest is not in an ez. Allow the rename.
265          return;
266        }
267        throw new IOException(
268            src + " can't be moved from an encryption zone.");
269      }
270    } else {
271      if (dstInEZ) {
272        throw new IOException(
273            src + " can't be moved into an encryption zone.");
274      }
275    }
276
277    if (srcInEZ || dstInEZ) {
278      Preconditions.checkState(srcEZI != null, "couldn't find src EZ?");
279      Preconditions.checkState(dstEZI != null, "couldn't find dst EZ?");
280      if (srcEZI != dstEZI) {
281        final String srcEZPath = getFullPathName(srcEZI);
282        final String dstEZPath = getFullPathName(dstEZI);
283        final StringBuilder sb = new StringBuilder(src);
284        sb.append(" can't be moved from encryption zone ");
285        sb.append(srcEZPath);
286        sb.append(" to encryption zone ");
287        sb.append(dstEZPath);
288        sb.append(".");
289        throw new IOException(sb.toString());
290      }
291    }
292  }
293
294  @VisibleForTesting
295  void setAllowNestedEZ() {
296    allowNestedEZ = true;
297  }
298
299  @VisibleForTesting
300  void setDisallowNestedEZ() {
301    allowNestedEZ = false;
302  }
303
304  /**
305   * Create a new encryption zone.
306   * <p/>
307   * Called while holding the FSDirectory lock.
308   */
309  XAttr createEncryptionZone(String src, CipherSuite suite,
310      CryptoProtocolVersion version, String keyName)
311      throws IOException {
312    assert dir.hasWriteLock();
313    if (dir.isNonEmptyDirectory(src)) {
314      throw new IOException(
315          "Attempt to create an encryption zone for a non-empty directory.");
316    }
317
318    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
319    if (srcIIP != null &&
320        srcIIP.getLastINode() != null &&
321        !srcIIP.getLastINode().isDirectory()) {
322      throw new IOException("Attempt to create an encryption zone for a file.");
323    }
324    EncryptionZoneInt ezi = getEncryptionZoneForPath(srcIIP);
325    if (!allowNestedEZ && ezi != null) {
326      throw new IOException("Directory " + src + " is already in an " +
327          "encryption zone. (" + getFullPathName(ezi) + ")");
328    }
329
330    final HdfsProtos.ZoneEncryptionInfoProto proto =
331        PBHelper.convert(suite, version, keyName);
332    final XAttr ezXAttr = XAttrHelper
333        .buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, proto.toByteArray());
334
335    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
336    xattrs.add(ezXAttr);
337    // updating the xattr will call addEncryptionZone,
338    // done this way to handle edit log loading
339    dir.unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
340    return ezXAttr;
341  }
342
343  /**
344   * Cursor-based listing of encryption zones.
345   * <p/>
346   * Called while holding the FSDirectory lock.
347   */
348  BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
349      throws IOException {
350    assert dir.hasReadLock();
351    if (!hasCreatedEncryptionZone()) {
352      final List<EncryptionZone> emptyZones = Lists.newArrayList();
353      return new BatchedListEntries<EncryptionZone>(emptyZones, false);
354    }
355    NavigableMap<Long, EncryptionZoneInt> tailMap = encryptionZones.tailMap
356        (prevId, false);
357    final int numResponses = Math.min(maxListEncryptionZonesResponses,
358        tailMap.size());
359    final List<EncryptionZone> zones =
360        Lists.newArrayListWithExpectedSize(numResponses);
361
362    int count = 0;
363    for (EncryptionZoneInt ezi : tailMap.values()) {
364      /*
365       Skip EZs that are only present in snapshots. Re-resolve the path to 
366       see if the path's current inode ID matches EZ map's INode ID.
367       
368       INode#getFullPathName simply calls getParent recursively, so will return
369       the INode's parents at the time it was snapshotted. It will not 
370       contain a reference INode.
371      */
372      final String pathName = getFullPathName(ezi);
373      INodesInPath iip = dir.getINodesInPath(pathName, false);
374      INode lastINode = iip.getLastINode();
375      if (lastINode == null || lastINode.getId() != ezi.getINodeId()) {
376        continue;
377      }
378      // Add the EZ to the result list
379      zones.add(new EncryptionZone(ezi.getINodeId(), pathName,
380          ezi.getSuite(), ezi.getVersion(), ezi.getKeyName()));
381      count++;
382      if (count >= numResponses) {
383        break;
384      }
385    }
386    final boolean hasMore = (numResponses < tailMap.size());
387    return new BatchedListEntries<EncryptionZone>(zones, hasMore);
388  }
389
390  /**
391   * @return Whether there has been any attempt to create an encryption zone in
392   * the cluster at all. If not, it is safe to quickly return null when
393   * checking the encryption information of any file or directory in the
394   * cluster.
395   */
396  public boolean hasCreatedEncryptionZone() {
397    return encryptionZones != null;
398  }
399}