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     */
018    package org.apache.hadoop.hdfs.server.namenode;
019    
020    import java.io.IOException;
021    import java.util.EnumSet;
022    import java.util.List;
023    import java.util.NavigableMap;
024    import java.util.TreeMap;
025    
026    import com.google.common.base.Preconditions;
027    import com.google.common.collect.Lists;
028    import org.apache.hadoop.conf.Configuration;
029    import org.apache.hadoop.crypto.CipherSuite;
030    import org.apache.hadoop.crypto.CryptoProtocolVersion;
031    import org.apache.hadoop.fs.UnresolvedLinkException;
032    import org.apache.hadoop.fs.XAttr;
033    import org.apache.hadoop.fs.XAttrSetFlag;
034    import org.apache.hadoop.hdfs.DFSConfigKeys;
035    import org.apache.hadoop.hdfs.XAttrHelper;
036    import org.apache.hadoop.hdfs.protocol.EncryptionZone;
037    import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
038    import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
039    import org.apache.hadoop.hdfs.protocolPB.PBHelper;
040    import org.slf4j.Logger;
041    import org.slf4j.LoggerFactory;
042    
043    
044    import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
045    import 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     */
055    public 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    }