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.hbase.io.asyncfs; 019 020import java.io.IOException; 021import org.apache.hadoop.fs.CommonConfigurationKeysPublic; 022import org.apache.hadoop.fs.FSDataOutputStream; 023import org.apache.hadoop.fs.FileSystem; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.fs.StreamCapabilities; 026import org.apache.hadoop.hbase.util.CommonFSUtils; 027import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; 028import org.apache.hadoop.hdfs.DistributedFileSystem; 029import org.apache.yetus.audience.InterfaceAudience; 030 031import org.apache.hbase.thirdparty.com.google.common.io.Closeables; 032import org.apache.hbase.thirdparty.io.netty.channel.Channel; 033import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; 034 035/** 036 * Helper class for creating AsyncFSOutput. 037 */ 038@InterfaceAudience.Private 039public final class AsyncFSOutputHelper { 040 041 private AsyncFSOutputHelper() { 042 } 043 044 /** 045 * Create {@link FanOutOneBlockAsyncDFSOutput} for {@link DistributedFileSystem}, and a simple 046 * implementation for other {@link FileSystem} which wraps around a {@link FSDataOutputStream}. 047 */ 048 public static AsyncFSOutput createOutput(FileSystem fs, Path f, boolean overwrite, 049 boolean createParent, short replication, long blockSize, EventLoopGroup eventLoopGroup, 050 Class<? extends Channel> channelClass) 051 throws IOException, CommonFSUtils.StreamLacksCapabilityException { 052 if (fs instanceof DistributedFileSystem) { 053 return FanOutOneBlockAsyncDFSOutputHelper.createOutput((DistributedFileSystem) fs, f, 054 overwrite, createParent, replication, blockSize, eventLoopGroup, channelClass); 055 } 056 final FSDataOutputStream out; 057 int bufferSize = fs.getConf().getInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, 058 CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT); 059 // This is not a Distributed File System, so it won't be erasure coded; no builder API needed 060 if (createParent) { 061 out = fs.create(f, overwrite, bufferSize, replication, blockSize, null); 062 } else { 063 out = fs.createNonRecursive(f, overwrite, bufferSize, replication, blockSize, null); 064 } 065 // After we create the stream but before we attempt to use it at all 066 // ensure that we can provide the level of data safety we're configured 067 // to provide. 068 if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true)) { 069 if (!out.hasCapability(StreamCapabilities.HFLUSH)) { 070 Closeables.close(out, true); 071 throw new StreamLacksCapabilityException(StreamCapabilities.HFLUSH); 072 } 073 if (!out.hasCapability(StreamCapabilities.HSYNC)) { 074 Closeables.close(out, true); 075 throw new StreamLacksCapabilityException(StreamCapabilities.HSYNC); 076 } 077 } 078 return new WrapperAsyncFSOutput(f, out); 079 } 080}