Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -120,4 +120,16 @@
<name>fs.contract.supports-unix-permissions</name>
<value>false</value>
</property>

<property>
<name>fs.contract.supports-hsync</name>
<value>false</value>
<!-- TODO: switch to true after HDDS-7688 -->
</property>

<property>
<name>fs.contract.supports-hflush</name>
<value>false</value>
<!-- TODO: switch to true after HDDS-7688 -->
</property>
</configuration>
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
Expand Down Expand Up @@ -278,8 +279,14 @@ private FSDataOutputStream createOutputStream(String key, short replication,
return new FSDataOutputStream(adapter.createStreamFile(key,
replication, overwrite, recursive), statistics);
}
return new FSDataOutputStream(adapter.createFile(key,
replication, overwrite, recursive), statistics);
return new FSDataOutputStream(createFSOutputStream(
adapter.createFile(key,
replication, overwrite, recursive)), statistics);
}

protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return outputStream;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
Expand Down Expand Up @@ -259,8 +260,14 @@ private FSDataOutputStream createOutputStream(String key, short replication,
return new FSDataOutputStream(adapter.createStreamFile(key,
replication, overwrite, recursive), statistics);
}
return new FSDataOutputStream(adapter.createFile(key,
replication, overwrite, recursive), statistics);
return new FSDataOutputStream(createFSOutputStream(
adapter.createFile(key,
replication, overwrite, recursive)), statistics);
}

protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return outputStream;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.fs.ozone;

import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.impl.StoreImplementationUtils;
import org.apache.hadoop.ozone.client.io.ECKeyOutputStream;
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.util.StringUtils;

import java.io.OutputStream;

/**
* This class is used to workaround Hadoop2 compatibility issues.
*
* Hadoop 2 does not support StreamCapabilities, so we create different modules
* for Hadoop2 and Hadoop3 profiles.
*
* The OzoneFileSystem and RootedOzoneFileSystem in Hadoop3 profile uses
* CapableOzoneFSOutputStream which implements StreamCapabilities interface,
* whereas the ones in Hadoop2 profile does not.
*/
public class CapableOzoneFSOutputStream extends OzoneFSOutputStream
implements StreamCapabilities {
public CapableOzoneFSOutputStream(OzoneFSOutputStream outputStream) {
super(outputStream.getWrappedOutputStream());
}

@Override
public boolean hasCapability(String capability) {
OutputStream os = getWrappedOutputStream().getOutputStream();
if (os instanceof ECKeyOutputStream) {
return false;
} else if (os instanceof KeyOutputStream) {
switch (StringUtils.toLowerCase(capability)) {
case StreamCapabilities.HFLUSH:
case StreamCapabilities.HSYNC:
// TODO: switch to true after HDDS-7688
return false;
default:
return false;
}
}
// deal with CryptoOutputStream
return StoreImplementationUtils.hasCapability(os, capability);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@
* TODO: Make outputStream generic for both rest and rpc clients
* This class is not thread safe.
*/
public class OzoneFSOutputStream extends OutputStream implements Syncable {
public class OzoneFSOutputStream extends OutputStream
implements Syncable {

private final OzoneOutputStream outputStream;

Expand Down Expand Up @@ -68,4 +69,10 @@ public void hflush() throws IOException {
public void hsync() throws IOException {
outputStream.hsync();
}

protected OzoneOutputStream getWrappedOutputStream() {
return outputStream;
}


}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;

import org.apache.hadoop.crypto.key.KeyProvider;
Expand Down Expand Up @@ -100,4 +101,10 @@ protected OzoneClientAdapter createAdapter(ConfigurationSource conf,
protected InputStream createFSInputStream(InputStream inputStream) {
return new CapableOzoneFSInputStream(inputStream, statistics);
}

@Override
protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return new CapableOzoneFSOutputStream(outputStream);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;

/**
Expand Down Expand Up @@ -98,4 +99,10 @@ protected OzoneClientAdapter createAdapter(ConfigurationSource conf,
protected InputStream createFSInputStream(InputStream inputStream) {
return new CapableOzoneFSInputStream(inputStream, statistics);
}

@Override
protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return new CapableOzoneFSOutputStream(outputStream);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;

import org.apache.hadoop.crypto.key.KeyProvider;
Expand Down Expand Up @@ -100,4 +101,10 @@ protected OzoneClientAdapter createAdapter(ConfigurationSource conf,
protected InputStream createFSInputStream(InputStream inputStream) {
return new CapableOzoneFSInputStream(inputStream, statistics);
}

@Override
protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return new CapableOzoneFSOutputStream(outputStream);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;

/**
Expand Down Expand Up @@ -98,4 +99,10 @@ protected OzoneClientAdapter createAdapter(ConfigurationSource conf,
protected InputStream createFSInputStream(InputStream inputStream) {
return new CapableOzoneFSInputStream(inputStream, statistics);
}

@Override
protected OutputStream createFSOutputStream(
OzoneFSOutputStream outputStream) {
return new CapableOzoneFSOutputStream(outputStream);
}
}