Skip to content

HADOOP-16854. ABFS: Fix for the OutOfMemoryException in AbfsOutputStream #2014

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 6 commits into from
Closed
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
12 changes: 12 additions & 0 deletions hadoop-tools/hadoop-azure/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,12 @@
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. can you add to hadoop project pom and then refer here. its how we guarantee consistent versions.
  2. do we really need to add a new JAR into production just for annotations? if that is all it is for, maybe we could somehow avoid doing that

which annotations is it actually for? as VisibleForTesting is in guava

<groupId>com.google.code.findbugs</groupId>
<artifactId>annotations</artifactId>
<version>3.0.1</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.eclipse.jetty</groupId>
Expand Down Expand Up @@ -288,6 +294,12 @@
<artifactId>hamcrest-library</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>net.jodah</groupId>
<artifactId>concurrentunit</artifactId>
<version>0.4.6</version>
<scope>test</scope>
</dependency>

</dependencies>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,10 +130,20 @@ public class AbfsConfiguration{
DefaultValue = AZURE_BLOCK_LOCATION_HOST_DEFAULT)
private String azureBlockLocationHost;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
@BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_SHOULD_USE_OLD_ABFSOUTPUTSTREAM,
DefaultValue = DEFAULT_AZURE_SHOULD_USE_OLD_ABFSOUTPUTSTREAM)
private boolean shouldUseOlderAbfsOutputStream;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_CONCURRENCY_FACTOR,
MinValue = 1,
DefaultValue = MAX_CONCURRENT_WRITE_THREADS)
private int maxConcurrentWriteThreads;
DefaultValue = DEFAULT_WRITE_CONCURRENCY_FACTOR)
private int writeConcurrencyFactor;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_WRITE_MEM_USAGE_PERCENTAGE,
MinValue = MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE,
MaxValue = MAX_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE,
DefaultValue = DEFAULT_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE)
private int maxWriteMemoryUsagePercentage;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_LIST_MAX_RESULTS,
MinValue = 1,
Expand Down Expand Up @@ -464,8 +474,16 @@ public String getAzureBlockLocationHost() {
return this.azureBlockLocationHost;
}

public int getMaxConcurrentWriteThreads() {
return this.maxConcurrentWriteThreads;
public boolean shouldUseOlderAbfsOutputStream() {
return this.shouldUseOlderAbfsOutputStream;
}

public int getWriteConcurrencyFactor() {
return this.writeConcurrencyFactor;
}

public int getMaxWriteMemoryUsagePercentage() {
return this.maxWriteMemoryUsagePercentage;
}

public int getMaxConcurrentReadThreads() {
Expand Down Expand Up @@ -784,4 +802,10 @@ private String appendSlashIfNeeded(String authority) {
return authority;
}

@VisibleForTesting
void setShouldUseOlderAbfsOutputStream(
boolean shouldUseOlderAbfsOutputStream) {
this.shouldUseOlderAbfsOutputStream = shouldUseOlderAbfsOutputStream;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamContext;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamOld;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamContext;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStreamStatisticsImpl;
import org.apache.hadoop.fs.azurebfs.services.AbfsPermission;
Expand Down Expand Up @@ -424,6 +425,15 @@ public OutputStream createFile(final Path path,
isNamespaceEnabled ? getOctalNotation(umask) : null);
perfInfo.registerResult(op.getResult()).registerSuccess(true);

if (abfsConfiguration.shouldUseOlderAbfsOutputStream()) {
return new AbfsOutputStreamOld(
client,
statistics,
getRelativePath(path),
0,
populateAbfsOutputStreamContext());
}

return new AbfsOutputStream(
client,
statistics,
Expand All @@ -439,6 +449,10 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext() {
.enableFlush(abfsConfiguration.isFlushEnabled())
.disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled())
.withStreamStatistics(new AbfsOutputStreamStatisticsImpl())
.withWriteConcurrencyFactor(abfsConfiguration
.getWriteConcurrencyFactor())
.withMaxWriteMemoryUsagePercentage(abfsConfiguration
.getMaxWriteMemoryUsagePercentage())
.build();
}

Expand Down Expand Up @@ -530,6 +544,16 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic

perfInfo.registerSuccess(true);


if (abfsConfiguration.shouldUseOlderAbfsOutputStream()) {
return new AbfsOutputStreamOld(
client,
statistics,
getRelativePath(path),
0,
populateAbfsOutputStreamContext());
}

return new AbfsOutputStream(
client,
statistics,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@ public final class ConfigurationKeys {
public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
public static final String AZURE_SHOULD_USE_OLD_ABFSOUTPUTSTREAM = "fs.azure.use.olderabfsoutputstream";
public static final String AZURE_WRITE_CONCURRENCY_FACTOR = "fs.azure.write.concurrency.factor";
public static final String AZURE_MAX_WRITE_MEM_USAGE_PERCENTAGE = "fs.azure.max.write.memory.usage.percentage";
public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in";
public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
public static final String AZURE_LIST_MAX_RESULTS = "fs.azure.list.max.results";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,11 @@ public final class FileSystemConfigurations {

public static final int MAX_CONCURRENT_READ_THREADS = 12;
public static final int MAX_CONCURRENT_WRITE_THREADS = 8;
public static final int DEFAULT_WRITE_CONCURRENCY_FACTOR = 4;
public static final boolean DEFAULT_AZURE_SHOULD_USE_OLD_ABFSOUTPUTSTREAM = false;
public static final int MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE = 20;
public static final int MAX_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE = 90;
public static final int DEFAULT_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE = MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE;
public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false;
public static final boolean DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = false;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,161 @@
/**
* 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.azurebfs.services;

import java.util.concurrent.ArrayBlockingQueue;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;

import static java.lang.Math.ceil;
import static java.lang.Math.min;

import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE;

/**
* Pool for byte[]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

. at ehd for javadoc in java 8

*/
public class AbfsByteBufferPool {

private static final int HUNDRED = 100;

/**
* Queue holding the free buffers.
*/
private ArrayBlockingQueue<byte[]> freeBuffers;
/**
* Count to track the buffers issued from AbfsByteBufferPool and yet to be
* returned.
*/
private int numBuffersInUse;

private int bufferSize;

private int maxBuffersToPool;
private int maxMemUsagePercentage;

private static final Runtime RT = Runtime.getRuntime();
private static final int AVAILABLE_PROCESSORS = RT.availableProcessors();

private static int MAX_BUFFERS_THAT_CAN_BE_IN_USE;

/**
* @param bufferSize Size of the byte[] to be returned.
* @param queueCapacity Maximum number of buffers that the pool can
* keep within the pool.
* @param maxMemUsagePercentage Maximum percentage of memory that can
* be used by the pool from the max
* available memory.
*/
public AbfsByteBufferPool(final int bufferSize, final int queueCapacity,
final int maxMemUsagePercentage) {
validate(queueCapacity, maxMemUsagePercentage);
this.maxMemUsagePercentage = maxMemUsagePercentage;
this.bufferSize = bufferSize;
this.numBuffersInUse = 0;
this.maxBuffersToPool = queueCapacity;
freeBuffers = new ArrayBlockingQueue<>(queueCapacity);
}

private void validate(final int queueCapacity,
final int maxWriteMemUsagePercentage) {
Preconditions.checkArgument(maxWriteMemUsagePercentage
>= MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE
&& maxWriteMemUsagePercentage
<= MAX_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE,
"maxWriteMemUsagePercentage should be in range (%s - %s)",
MIN_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE,
MAX_VALUE_MAX_AZURE_WRITE_MEM_USAGE_PERCENTAGE);
Preconditions
.checkArgument(queueCapacity > 0, "queueCapacity cannot be < 1");
}

private void setMaxBuffersThatCanBeInUse() {
double freeMemory = RT.maxMemory() - (RT.totalMemory() - RT.freeMemory());
int bufferCountByMemory = (int) ceil(
(freeMemory * maxMemUsagePercentage / HUNDRED) / bufferSize);
int bufferCountByMaxFreeBuffers = maxBuffersToPool + AVAILABLE_PROCESSORS;
MAX_BUFFERS_THAT_CAN_BE_IN_USE = min(bufferCountByMemory,
bufferCountByMaxFreeBuffers);
if (MAX_BUFFERS_THAT_CAN_BE_IN_USE < 2) {
MAX_BUFFERS_THAT_CAN_BE_IN_USE = 2;
}
}

private synchronized boolean isPossibleToIssueNewBuffer() {
setMaxBuffersThatCanBeInUse();
return numBuffersInUse < MAX_BUFFERS_THAT_CAN_BE_IN_USE;
}

/**
* @return byte[] from the pool if available otherwise new byte[] is returned.
* Waits if pool is empty and already maximum number of buffers are in use.
*/
public byte[] get() {
byte[] byteArray = null;
synchronized (this) {
byteArray = freeBuffers.poll();
if (byteArray == null && isPossibleToIssueNewBuffer()) {
byteArray = new byte[bufferSize];
}
if (byteArray != null) {
numBuffersInUse++;
return byteArray;
}
}
try {
byteArray = freeBuffers.take();
synchronized (this) {
numBuffersInUse++;
return byteArray;
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
}
}

/**
* @param byteArray The buffer to be offered back to the pool.
*/
@SuppressFBWarnings("RV_RETURN_VALUE_IGNORED")
public synchronized void release(byte[] byteArray) {
Preconditions.checkArgument(byteArray.length == bufferSize,
"Buffer size has" + " to be %s (Received buffer length: %s)",
bufferSize, byteArray.length);
numBuffersInUse--;
if (numBuffersInUse < 0) {
numBuffersInUse = 0;
}
freeBuffers.offer(byteArray);
}

@VisibleForTesting
public synchronized int getBuffersInUse() {
return this.numBuffersInUse;
}

@VisibleForTesting
public synchronized ArrayBlockingQueue<byte[]> getFreeBuffers() {
return freeBuffers;
}

}
Loading