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