, org.apache.hadoop.io.serial.RawComparator {
+ @Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2);
}
diff --git a/src/java/org/apache/hadoop/io/SecureIOUtils.java b/src/java/org/apache/hadoop/io/SecureIOUtils.java
new file mode 100644
index 0000000000000..321b0e89db66e
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/SecureIOUtils.java
@@ -0,0 +1,208 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.io;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.nativeio.Errno;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.io.nativeio.NativeIOException;
+import org.apache.hadoop.io.nativeio.NativeIO.Stat;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * This class provides secure APIs for opening and creating files on the local
+ * disk. The main issue this class tries to handle is that of symlink traversal.
+ *
+ * An example of such an attack is:
+ *
+ * - Malicious user removes his task's syslog file, and puts a link to the
+ * jobToken file of a target user.
+ * - Malicious user tries to open the syslog file via the servlet on the
+ * tasktracker.
+ * - The tasktracker is unaware of the symlink, and simply streams the contents
+ * of the jobToken file. The malicious user can now access potentially sensitive
+ * map outputs, etc. of the target user's job.
+ *
+ * A similar attack is possible involving task log truncation, but in that case
+ * due to an insecure write to a file.
+ *
+ */
+public class SecureIOUtils {
+
+ /**
+ * Ensure that we are set up to run with the appropriate native support code.
+ * If security is disabled, and the support code is unavailable, this class
+ * still tries its best to be secure, but is vulnerable to some race condition
+ * attacks.
+ *
+ * If security is enabled but the support code is unavailable, throws a
+ * RuntimeException since we don't want to run insecurely.
+ */
+ static {
+ boolean shouldBeSecure = UserGroupInformation.isSecurityEnabled();
+ boolean canBeSecure = NativeIO.isAvailable();
+
+ if (!canBeSecure && shouldBeSecure) {
+ throw new RuntimeException(
+ "Secure IO is not possible without native code extensions.");
+ }
+
+ // Pre-cache an instance of the raw FileSystem since we sometimes
+ // do secure IO in a shutdown hook, where this call could fail.
+ try {
+ rawFilesystem = FileSystem.getLocal(new Configuration()).getRaw();
+ } catch (IOException ie) {
+ throw new RuntimeException(
+ "Couldn't obtain an instance of RawLocalFileSystem.");
+ }
+
+ // SecureIO just skips security checks in the case that security is
+ // disabled
+ skipSecurity = !canBeSecure;
+ }
+
+ private final static boolean skipSecurity;
+ private final static FileSystem rawFilesystem;
+
+ /**
+ * Open the given File for read access, verifying the expected user/group
+ * constraints.
+ * @param f the file that we are trying to open
+ * @param expectedOwner the expected user owner for the file
+ * @param expectedGroup the expected group owner for the file
+ * @throws IOException if an IO Error occurred, or the user/group does not
+ * match
+ */
+ public static FileInputStream openForRead(File f, String expectedOwner,
+ String expectedGroup) throws IOException {
+ if (skipSecurity) {
+ // Subject to race conditions but this is the best we can do
+ FileStatus status =
+ rawFilesystem.getFileStatus(new Path(f.getAbsolutePath()));
+ checkStat(f, status.getOwner(), status.getGroup(),
+ expectedOwner, expectedGroup);
+ return new FileInputStream(f);
+ }
+
+ FileInputStream fis = new FileInputStream(f);
+ boolean success = false;
+ try {
+ Stat stat = NativeIO.fstat(fis.getFD());
+ checkStat(f, stat.getOwner(), stat.getGroup(), expectedOwner,
+ expectedGroup);
+ success = true;
+ return fis;
+ } finally {
+ if (!success) {
+ fis.close();
+ }
+ }
+ }
+
+ private static FileOutputStream insecureCreateForWrite(File f,
+ int permissions) throws IOException {
+ // If we can't do real security, do a racy exists check followed by an
+ // open and chmod
+ if (f.exists()) {
+ throw new AlreadyExistsException("File " + f + " already exists");
+ }
+ FileOutputStream fos = new FileOutputStream(f);
+ boolean success = false;
+ try {
+ rawFilesystem.setPermission(new Path(f.getAbsolutePath()),
+ new FsPermission((short)permissions));
+ success = true;
+ return fos;
+ } finally {
+ if (!success) {
+ fos.close();
+ }
+ }
+ }
+
+ /**
+ * Open the specified File for write access, ensuring that it does not exist.
+ * @param f the file that we want to create
+ * @param permissions we want to have on the file (if security is enabled)
+ *
+ * @throws AlreadyExistsException if the file already exists
+ * @throws IOException if any other error occurred
+ */
+ public static FileOutputStream createForWrite(File f, int permissions)
+ throws IOException {
+ if (skipSecurity) {
+ return insecureCreateForWrite(f, permissions);
+ } else {
+ // Use the native wrapper around open(2)
+ try {
+ FileDescriptor fd = NativeIO.open(f.getAbsolutePath(),
+ NativeIO.O_WRONLY | NativeIO.O_CREAT | NativeIO.O_EXCL,
+ permissions);
+ return new FileOutputStream(fd);
+ } catch (NativeIOException nioe) {
+ if (nioe.getErrno() == Errno.EEXIST) {
+ throw new AlreadyExistsException(nioe);
+ }
+ throw nioe;
+ }
+ }
+ }
+
+ private static void checkStat(File f, String owner, String group,
+ String expectedOwner,
+ String expectedGroup) throws IOException {
+ if (expectedOwner != null &&
+ !expectedOwner.equals(owner)) {
+ throw new IOException(
+ "Owner '" + owner + "' for path " + f + " did not match " +
+ "expected owner '" + expectedOwner + "'");
+ }
+ if (expectedGroup != null &&
+ !expectedGroup.equals(group)) {
+ throw new IOException(
+ "Group '" + group + "' for path " + f + " did not match " +
+ "expected group '" + expectedGroup + "'");
+ }
+ }
+
+ /**
+ * Signals that an attempt to create a file at a given pathname has failed
+ * because another file already existed at that path.
+ */
+ public static class AlreadyExistsException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ public AlreadyExistsException(String msg) {
+ super(msg);
+ }
+
+ public AlreadyExistsException(Throwable cause) {
+ super(cause);
+ }
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/SequenceFile.java b/src/java/org/apache/hadoop/io/SequenceFile.java
index b3f1630dd4952..91aed3257d84f 100644
--- a/src/java/org/apache/hadoop/io/SequenceFile.java
+++ b/src/java/org/apache/hadoop/io/SequenceFile.java
@@ -34,12 +34,14 @@
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.io.compress.zlib.ZlibFactory;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.TypedSerialization;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.*;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.ReflectionUtils;
@@ -93,10 +95,16 @@
* version number (e.g. SEQ4 or SEQ6)
*
*
- * keyClassName -key class
+ * key serialization name
*
*
- * valueClassName - value class
+ * key serialization configuration
+ *
+ *
+ * value serialization name
+ *
+ *
+ * value serialization data
*
*
* compression - A boolean which specifies if compression is turned on for
@@ -134,7 +142,7 @@
*
*
*
- * A sync-marker every few 100 bytes or so.
+ * A sync-marker every 2000 bytes or so.
*
*
*
@@ -153,7 +161,7 @@
*
*
*
- * A sync-marker every few 100 bytes or so.
+ * A sync-marker every 2000 bytes or so.
*
*
*
@@ -165,6 +173,7 @@
*
* Record Block
*
+ * - sync-marker
* - Compressed key-lengths block-size
* - Compressed key-lengths block
* - Compressed keys block-size
@@ -175,9 +184,6 @@
* - Compressed values block
*
*
- *
- * A sync-marker every few 100 bytes or so.
- *
*
*
* The compressed blocks of key lengths and value lengths consist of the
@@ -196,8 +202,9 @@ private SequenceFile() {} // no public ctor
private static final byte BLOCK_COMPRESS_VERSION = (byte)4;
private static final byte CUSTOM_COMPRESS_VERSION = (byte)5;
private static final byte VERSION_WITH_METADATA = (byte)6;
+ private static final byte SERIALIZATION_VERSION = (byte) 7;
private static byte[] VERSION = new byte[] {
- (byte)'S', (byte)'E', (byte)'Q', VERSION_WITH_METADATA
+ (byte)'S', (byte)'E', (byte)'Q', SERIALIZATION_VERSION
};
private static final int SYNC_ESCAPE = -1; // "length" of sync entries
@@ -285,6 +292,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass) throws IOException {
@@ -306,6 +314,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
@@ -330,6 +339,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, CompressionType compressionType,
@@ -355,6 +365,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, CompressionType compressionType,
@@ -381,6 +392,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
@@ -413,6 +425,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, int bufferSize,
@@ -444,6 +457,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
@@ -471,6 +485,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(Configuration conf, FSDataOutputStream out,
Class keyClass, Class valClass,
@@ -495,6 +510,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts
* @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public static Writer
createWriter(Configuration conf, FSDataOutputStream out,
@@ -527,18 +543,44 @@ public void writeCompressedBytes(DataOutputStream outStream)
* Size of stored data.
*/
public int getSize();
+
}
+
+ /**
+ * Make an InputStream from a ValueBytes.
+ * @param bytes the bytes to provide as input
+ * @return a new input stream with the bytes
+ * @throws IOException
+ */
+ private static InputStream readUncompressedBytes(ValueBytes bytes
+ ) throws IOException {
+ DataInputBuffer result = new DataInputBuffer();
+ if (bytes instanceof UncompressedBytes) {
+ MutableValueBytes concrete = (MutableValueBytes) bytes;
+ result.reset(concrete.data, concrete.dataSize);
+ } else {
+ DataOutputBuffer outBuf = new DataOutputBuffer();
+ bytes.writeUncompressedBytes(outBuf);
+ result.reset(outBuf.getData(), outBuf.getLength());
+ }
+ return result;
+ }
+
- private static class UncompressedBytes implements ValueBytes {
- private int dataSize;
- private byte[] data;
-
- private UncompressedBytes() {
+ private static abstract class MutableValueBytes implements ValueBytes {
+ protected byte[] data;
+ protected int dataSize;
+
+ MutableValueBytes() {
data = null;
dataSize = 0;
}
+
+ public int getSize() {
+ return dataSize;
+ }
- private void reset(DataInputStream in, int length) throws IOException {
+ void reset(DataInputStream in, int length) throws IOException {
if (data == null) {
data = new byte[length];
} else if (length > data.length) {
@@ -548,10 +590,14 @@ private void reset(DataInputStream in, int length) throws IOException {
in.readFully(data, 0, length);
dataSize = length;
}
-
- public int getSize() {
- return dataSize;
+
+ void set(MutableValueBytes other) {
+ data = other.data;
+ dataSize = other.dataSize;
}
+ }
+
+ private static class UncompressedBytes extends MutableValueBytes {
public void writeUncompressedBytes(DataOutputStream outStream)
throws IOException {
@@ -566,34 +612,15 @@ public void writeCompressedBytes(DataOutputStream outStream)
} // UncompressedBytes
- private static class CompressedBytes implements ValueBytes {
- private int dataSize;
- private byte[] data;
+ private static class CompressedBytes extends MutableValueBytes {
DataInputBuffer rawData = null;
CompressionCodec codec = null;
CompressionInputStream decompressedStream = null;
private CompressedBytes(CompressionCodec codec) {
- data = null;
- dataSize = 0;
this.codec = codec;
}
- private void reset(DataInputStream in, int length) throws IOException {
- if (data == null) {
- data = new byte[length];
- } else if (length > data.length) {
- data = new byte[Math.max(length, data.length * 2)];
- }
- dataSize = -1;
- in.readFully(data, 0, length);
- dataSize = length;
- }
-
- public int getSize() {
- return dataSize;
- }
-
public void writeUncompressedBytes(DataOutputStream outStream)
throws IOException {
if (decompressedStream == null) {
@@ -738,9 +765,6 @@ public static class Writer implements java.io.Closeable {
boolean ownOutputStream = true;
DataOutputBuffer buffer = new DataOutputBuffer();
- Class keyClass;
- Class valClass;
-
private final CompressionType compress;
CompressionCodec codec = null;
CompressionOutputStream deflateFilter = null;
@@ -748,9 +772,8 @@ public static class Writer implements java.io.Closeable {
Metadata metadata = null;
Compressor compressor = null;
- protected Serializer keySerializer;
- protected Serializer uncompressedValSerializer;
- protected Serializer compressedValSerializer;
+ protected Serialization
* @param
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.Serialization}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public interface Serializer {
/**
* Prepare the serializer for writing.
diff --git a/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java b/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
index 8511d25bcde65..30e773c0e7729 100644
--- a/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
@@ -23,7 +23,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -36,9 +35,13 @@
* A {@link Serialization} for {@link Writable}s that delegates to
* {@link Writable#write(java.io.DataOutput)} and
* {@link Writable#readFields(java.io.DataInput)}.
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.lib.WritableSerialization}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class WritableSerialization extends Configured
implements Serialization {
static class WritableDeserializer extends Configured
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
index 4a748223e5fc5..49fdf6e2810a9 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
@@ -25,9 +25,13 @@
* Tag interface for Avro 'reflect' serializable classes. Classes implementing
* this interface can be serialized/deserialized using
* {@link AvroReflectSerialization}.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroReflectSerializable}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
-public interface AvroReflectSerializable {
+@Deprecated
+public interface AvroReflectSerializable
+ extends org.apache.hadoop.io.serial.lib.avro.AvroReflectSerializable {
}
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
index 5636b597e33a4..2c24144e203d7 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
@@ -35,11 +35,13 @@
* serialization, it must either be in the package list configured via
* avro.reflect.pkgs or implement
* {@link AvroReflectSerializable} interface.
- *
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@SuppressWarnings("unchecked")
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class AvroReflectSerialization extends AvroSerialization{
/**
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
index 27bb255696db6..402a0ad64542e 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
@@ -37,9 +37,12 @@
/**
* Base class for providing serialization to Avro types.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public abstract class AvroSerialization extends Configured
implements Serialization{
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
index e49d7a09dd53a..4ce8c72fe09c9 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
@@ -30,10 +30,13 @@
/**
* Serialization for Avro Specific classes. This serialization is to be used
* for classes generated by Avro's 'specific' compiler.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@SuppressWarnings("unchecked")
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class AvroSpecificSerialization
extends AvroSerialization{
diff --git a/src/java/org/apache/hadoop/io/serializer/package-info.java b/src/java/org/apache/hadoop/io/serializer/package-info.java
new file mode 100644
index 0000000000000..b7214457a0f54
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serializer/package-info.java
@@ -0,0 +1,41 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+/**
+ * This deprecated package provides a mechanism for using different
+ * serialization frameworks in Hadoop. The property "io.serializations"
+ * defines a list of
+ * {@link org.apache.hadoop.io.serializer.Serialization}s that know how to
+ * create {@link org.apache.hadoop.io.serializer.Serializer}s and
+ * {@link org.apache.hadoop.io.serializer.Deserializer}s.
+ *
+ *
+ * To add a new serialization framework write an implementation of
+ * {@link org.apache.hadoop.io.serializer.Serialization} and add its name to
+ * the "io.serializations" property.
+ *
+ *
+ * This package has been replaced by the {@link org.apache.hadoop.io.serial}
+ * package.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+@Deprecated
+package org.apache.hadoop.io.serializer;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/src/java/org/apache/hadoop/security/SaslRpcServer.java b/src/java/org/apache/hadoop/security/SaslRpcServer.java
index cc594b8d0e923..77a328e40f18b 100644
--- a/src/java/org/apache/hadoop/security/SaslRpcServer.java
+++ b/src/java/org/apache/hadoop/security/SaslRpcServer.java
@@ -239,7 +239,7 @@ public void handle(Callback[] callbacks) throws InvalidToken,
if (ac.isAuthorized()) {
if (LOG.isDebugEnabled()) {
String username = getIdentifier(authzid, secretManager).getUser()
- .getUserName().toString();
+ .getUserName();
LOG.debug("SASL server DIGEST-MD5 callback: setting "
+ "canonicalized client ID: " + username);
}
diff --git a/src/java/org/apache/hadoop/util/Options.java b/src/java/org/apache/hadoop/util/Options.java
index 23169e3af3533..88709561713e0 100644
--- a/src/java/org/apache/hadoop/util/Options.java
+++ b/src/java/org/apache/hadoop/util/Options.java
@@ -22,6 +22,8 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
/**
* This class allows generic access to variable length type-safe parameter
@@ -79,6 +81,16 @@ public long getValue() {
}
}
+ public static abstract class SerializationOption {
+ private final Serialization> value;
+ protected SerializationOption(Serialization> value) {
+ this.value = value;
+ }
+ public Serialization> getValue() {
+ return value;
+ }
+ }
+
public static abstract class PathOption {
private final Path value;
protected PathOption(Path value) {
@@ -119,6 +131,16 @@ public Progressable getValue() {
}
}
+ public static abstract class ComparatorOption {
+ private final RawComparator value;
+ protected ComparatorOption(RawComparator value) {
+ this.value = value;
+ }
+ public RawComparator getValue() {
+ return value;
+ }
+ }
+
/**
* Find the first option of the required class.
* @param the static class to find
@@ -129,8 +151,7 @@ public Progressable getValue() {
* @throws IOException
*/
@SuppressWarnings("unchecked")
- public static T getOption(Class cls, base [] opts
- ) throws IOException {
+ public static T getOption(Class cls, base [] opts) {
for(base o: opts) {
if (o.getClass() == cls) {
return (T) o;
diff --git a/src/java/org/apache/hadoop/util/ReflectionUtils.java b/src/java/org/apache/hadoop/util/ReflectionUtils.java
index 0387c7e8b8e0a..f9dac1fdd3ffb 100644
--- a/src/java/org/apache/hadoop/util/ReflectionUtils.java
+++ b/src/java/org/apache/hadoop/util/ReflectionUtils.java
@@ -37,9 +37,9 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.lib.WritableSerialization;
/**
* General reflection utils
@@ -49,7 +49,6 @@
public class ReflectionUtils {
private static final Class>[] EMPTY_ARRAY = new Class[]{};
- volatile private static SerializationFactory serialFactory = null;
/**
* Cache of constructors for each class. Pins the classes so they
@@ -257,43 +256,59 @@ protected synchronized CopyInCopyOutBuffer initialValue() {
}
};
- private static SerializationFactory getFactory(Configuration conf) {
- if (serialFactory == null) {
- serialFactory = new SerializationFactory(conf);
- }
- return serialFactory;
- }
-
/**
- * Make a copy of the writable object using serialization to a buffer
+ * Make a copy of the object using serialization to a buffer
* @param dst the object to copy from
* @param src the object to copy into, which is destroyed
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static T copy(Configuration conf,
- T src, T dst) throws IOException {
+ T src, T dst) throws IOException {
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Class cls = (Class) src.getClass();
+ Serialization serializer =
+ (Serialization) factory.getSerializationByType(cls);
+ return copy(conf, src, dst, serializer);
+ }
+
+ /**
+ * Make a copy of the object with the given serialization.
+ * @param the type to copy
+ * @param conf the configuration to initialize the new object with
+ * @param src the object to copy
+ * @param dst the object to copy into, which can be null
+ * @param serial the serialization to use
+ * @return the new object that was copied into
+ * @throws IOException
+ */
+ @SuppressWarnings("unchecked")
+ public static T copy(Configuration conf, T src, T dst,
+ Serialization serial) throws IOException {
CopyInCopyOutBuffer buffer = cloneBuffers.get();
buffer.outBuffer.reset();
- SerializationFactory factory = getFactory(conf);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
Class cls = (Class) src.getClass();
- Serializer serializer = factory.getSerializer(cls);
- serializer.open(buffer.outBuffer);
- serializer.serialize(src);
+ Serialization serializer =
+ (Serialization) factory.getSerializationByType(cls);
+ serializer.serialize(buffer.outBuffer, src);
buffer.moveData();
- Deserializer deserializer = factory.getDeserializer(cls);
- deserializer.open(buffer.inBuffer);
- dst = deserializer.deserialize(dst);
- return dst;
+ return serializer.deserialize(buffer.inBuffer, dst, conf);
}
+ private static Configuration defaultConfiguration = null;
+ private static synchronized Configuration getDefaultConfiguration() {
+ if (defaultConfiguration == null) {
+ defaultConfiguration = new Configuration();
+ }
+ return defaultConfiguration;
+ }
+
@Deprecated
public static void cloneWritableInto(Writable dst,
Writable src) throws IOException {
- CopyInCopyOutBuffer buffer = cloneBuffers.get();
- buffer.outBuffer.reset();
- src.write(buffer.outBuffer);
- buffer.moveData();
- dst.readFields(buffer.inBuffer);
+ WritableSerialization serial = new WritableSerialization();
+ serial.setSpecificType(src.getClass());
+ copy(getDefaultConfiguration(), src, dst, serial);
}
}
diff --git a/src/native/Makefile.am b/src/native/Makefile.am
index c731a6d4e4d69..b99e24a67b9c0 100644
--- a/src/native/Makefile.am
+++ b/src/native/Makefile.am
@@ -33,7 +33,8 @@ export PLATFORM = $(shell echo $$OS_NAME | tr [A-Z] [a-z])
AM_CPPFLAGS = @JNI_CPPFLAGS@ -I$(HADOOP_NATIVE_SRCDIR)/src \
-Isrc/org/apache/hadoop/io/compress/zlib \
- -Isrc/org/apache/hadoop/security
+ -Isrc/org/apache/hadoop/security \
+ -Isrc/org/apache/hadoop/io/nativeio/
AM_LDFLAGS = @JNI_LDFLAGS@ -m$(JVM_DATA_MODEL)
AM_CFLAGS = -g -Wall -fPIC -O2 -m$(JVM_DATA_MODEL)
@@ -41,8 +42,12 @@ lib_LTLIBRARIES = libhadoop.la
libhadoop_la_SOURCES = src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c \
src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c \
src/org/apache/hadoop/security/getGroup.c \
- src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c
-libhadoop_la_LDFLAGS = -version-info 1:0:0
+ src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c \
+ src/org/apache/hadoop/io/nativeio/file_descriptor.c \
+ src/org/apache/hadoop/io/nativeio/errno_enum.c \
+ src/org/apache/hadoop/io/nativeio/NativeIO.c
+
+libhadoop_la_LDFLAGS = -version-info 1:0:0 $(AM_LDFLAGS)
libhadoop_la_LIBADD = -ldl -ljvm
#
diff --git a/src/native/Makefile.in b/src/native/Makefile.in
index d71b8745a1042..554fcb8c567ea 100644
--- a/src/native/Makefile.in
+++ b/src/native/Makefile.in
@@ -93,7 +93,8 @@ libLTLIBRARIES_INSTALL = $(INSTALL)
LTLIBRARIES = $(lib_LTLIBRARIES)
libhadoop_la_DEPENDENCIES =
am_libhadoop_la_OBJECTS = ZlibCompressor.lo ZlibDecompressor.lo \
- getGroup.lo JniBasedUnixGroupsMapping.lo
+ getGroup.lo JniBasedUnixGroupsMapping.lo file_descriptor.lo \
+ errno_enum.lo NativeIO.lo
libhadoop_la_OBJECTS = $(am_libhadoop_la_OBJECTS)
DEFAULT_INCLUDES = -I. -I$(srcdir) -I.
depcomp = $(SHELL) $(top_srcdir)/config/depcomp
@@ -222,7 +223,8 @@ sysconfdir = @sysconfdir@
target_alias = @target_alias@
AM_CPPFLAGS = @JNI_CPPFLAGS@ -I$(HADOOP_NATIVE_SRCDIR)/src \
-Isrc/org/apache/hadoop/io/compress/zlib \
- -Isrc/org/apache/hadoop/security
+ -Isrc/org/apache/hadoop/security \
+ -Isrc/org/apache/hadoop/io/nativeio/
AM_LDFLAGS = @JNI_LDFLAGS@ -m$(JVM_DATA_MODEL)
AM_CFLAGS = -g -Wall -fPIC -O2 -m$(JVM_DATA_MODEL)
@@ -230,9 +232,12 @@ lib_LTLIBRARIES = libhadoop.la
libhadoop_la_SOURCES = src/org/apache/hadoop/io/compress/zlib/ZlibCompressor.c \
src/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.c \
src/org/apache/hadoop/security/getGroup.c \
- src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c
+ src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c \
+ src/org/apache/hadoop/io/nativeio/file_descriptor.c \
+ src/org/apache/hadoop/io/nativeio/errno_enum.c \
+ src/org/apache/hadoop/io/nativeio/NativeIO.c
-libhadoop_la_LDFLAGS = -version-info 1:0:0
+libhadoop_la_LDFLAGS = -version-info 1:0:0 $(AM_LDFLAGS)
libhadoop_la_LIBADD = -ldl -ljvm
all: config.h
$(MAKE) $(AM_MAKEFLAGS) all-am
@@ -326,8 +331,11 @@ distclean-compile:
-rm -f *.tab.c
@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/JniBasedUnixGroupsMapping.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/NativeIO.Plo@am__quote@
@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/ZlibCompressor.Plo@am__quote@
@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/ZlibDecompressor.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/errno_enum.Plo@am__quote@
+@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/file_descriptor.Plo@am__quote@
@AMDEP_TRUE@@am__include@ @am__quote@./$(DEPDIR)/getGroup.Plo@am__quote@
.c.o:
@@ -379,6 +387,27 @@ JniBasedUnixGroupsMapping.lo: src/org/apache/hadoop/security/JniBasedUnixGroupsM
@AMDEP_TRUE@@am__fastdepCC_FALSE@ DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
@am__fastdepCC_FALSE@ $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o JniBasedUnixGroupsMapping.lo `test -f 'src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c' || echo '$(srcdir)/'`src/org/apache/hadoop/security/JniBasedUnixGroupsMapping.c
+file_descriptor.lo: src/org/apache/hadoop/io/nativeio/file_descriptor.c
+@am__fastdepCC_TRUE@ if $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -MT file_descriptor.lo -MD -MP -MF "$(DEPDIR)/file_descriptor.Tpo" -c -o file_descriptor.lo `test -f 'src/org/apache/hadoop/io/nativeio/file_descriptor.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/file_descriptor.c; \
+@am__fastdepCC_TRUE@ then mv -f "$(DEPDIR)/file_descriptor.Tpo" "$(DEPDIR)/file_descriptor.Plo"; else rm -f "$(DEPDIR)/file_descriptor.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ source='src/org/apache/hadoop/io/nativeio/file_descriptor.c' object='file_descriptor.lo' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@ $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o file_descriptor.lo `test -f 'src/org/apache/hadoop/io/nativeio/file_descriptor.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/file_descriptor.c
+
+errno_enum.lo: src/org/apache/hadoop/io/nativeio/errno_enum.c
+@am__fastdepCC_TRUE@ if $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -MT errno_enum.lo -MD -MP -MF "$(DEPDIR)/errno_enum.Tpo" -c -o errno_enum.lo `test -f 'src/org/apache/hadoop/io/nativeio/errno_enum.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/errno_enum.c; \
+@am__fastdepCC_TRUE@ then mv -f "$(DEPDIR)/errno_enum.Tpo" "$(DEPDIR)/errno_enum.Plo"; else rm -f "$(DEPDIR)/errno_enum.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ source='src/org/apache/hadoop/io/nativeio/errno_enum.c' object='errno_enum.lo' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@ $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o errno_enum.lo `test -f 'src/org/apache/hadoop/io/nativeio/errno_enum.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/errno_enum.c
+
+NativeIO.lo: src/org/apache/hadoop/io/nativeio/NativeIO.c
+@am__fastdepCC_TRUE@ if $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -MT NativeIO.lo -MD -MP -MF "$(DEPDIR)/NativeIO.Tpo" -c -o NativeIO.lo `test -f 'src/org/apache/hadoop/io/nativeio/NativeIO.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/NativeIO.c; \
+@am__fastdepCC_TRUE@ then mv -f "$(DEPDIR)/NativeIO.Tpo" "$(DEPDIR)/NativeIO.Plo"; else rm -f "$(DEPDIR)/NativeIO.Tpo"; exit 1; fi
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ source='src/org/apache/hadoop/io/nativeio/NativeIO.c' object='NativeIO.lo' libtool=yes @AMDEPBACKSLASH@
+@AMDEP_TRUE@@am__fastdepCC_FALSE@ DEPDIR=$(DEPDIR) $(CCDEPMODE) $(depcomp) @AMDEPBACKSLASH@
+@am__fastdepCC_FALSE@ $(LIBTOOL) --tag=CC --mode=compile $(CC) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) $(AM_CPPFLAGS) $(CPPFLAGS) $(AM_CFLAGS) $(CFLAGS) -c -o NativeIO.lo `test -f 'src/org/apache/hadoop/io/nativeio/NativeIO.c' || echo '$(srcdir)/'`src/org/apache/hadoop/io/nativeio/NativeIO.c
+
mostlyclean-libtool:
-rm -f *.lo
diff --git a/src/native/config.h.in b/src/native/config.h.in
index 0fd92160bf8da..26f8b0fcfb243 100644
--- a/src/native/config.h.in
+++ b/src/native/config.h.in
@@ -3,6 +3,10 @@
/* The 'actual' dynamic-library for '-lz' */
#undef HADOOP_ZLIB_LIBRARY
+/* Define to 1 if you have the declaration of `strerror_r', and to 0 if you
+ don't. */
+#undef HAVE_DECL_STRERROR_R
+
/* Define to 1 if you have the header file. */
#undef HAVE_DLFCN_H
@@ -39,6 +43,9 @@
/* Define to 1 if you have the header file. */
#undef HAVE_STDLIB_H
+/* Define to 1 if you have the `strerror_r' function. */
+#undef HAVE_STRERROR_R
+
/* Define to 1 if you have the header file. */
#undef HAVE_STRINGS_H
@@ -81,8 +88,17 @@
/* Define to 1 if you have the ANSI C header files. */
#undef STDC_HEADERS
+/* Define to 1 if strerror_r returns char *. */
+#undef STRERROR_R_CHAR_P
+
/* Version number of package */
#undef VERSION
+/* Number of bits in a file offset, on hosts where this is settable. */
+#undef _FILE_OFFSET_BITS
+
+/* Define for large files, on AIX-style hosts. */
+#undef _LARGE_FILES
+
/* Define to empty if `const' does not conform to ANSI C. */
#undef const
diff --git a/src/native/configure b/src/native/configure
index 96bedbb04ebb2..c84776150dd13 100755
--- a/src/native/configure
+++ b/src/native/configure
@@ -464,7 +464,7 @@ ac_includes_default="\
# include
#endif"
-ac_subst_vars='SHELL PATH_SEPARATOR PACKAGE_NAME PACKAGE_TARNAME PACKAGE_VERSION PACKAGE_STRING PACKAGE_BUGREPORT exec_prefix prefix program_transform_name bindir sbindir libexecdir datadir sysconfdir sharedstatedir localstatedir libdir includedir oldincludedir infodir mandir build_alias host_alias target_alias DEFS ECHO_C ECHO_N ECHO_T LIBS INSTALL_PROGRAM INSTALL_SCRIPT INSTALL_DATA CYGPATH_W PACKAGE VERSION ACLOCAL AUTOCONF AUTOMAKE AUTOHEADER MAKEINFO install_sh STRIP ac_ct_STRIP INSTALL_STRIP_PROGRAM mkdir_p AWK SET_MAKE am__leading_dot AMTAR am__tar am__untar CC CFLAGS LDFLAGS CPPFLAGS ac_ct_CC EXEEXT OBJEXT DEPDIR am__include am__quote AMDEP_TRUE AMDEP_FALSE AMDEPBACKSLASH CCDEPMODE am__fastdepCC_TRUE am__fastdepCC_FALSE build build_cpu build_vendor build_os host host_cpu host_vendor host_os SED EGREP LN_S ECHO AR ac_ct_AR RANLIB ac_ct_RANLIB CPP CXX CXXFLAGS ac_ct_CXX CXXDEPMODE am__fastdepCXX_TRUE am__fastdepCXX_FALSE CXXCPP F77 FFLAGS ac_ct_F77 LIBTOOL JNI_LDFLAGS JNI_CPPFLAGS LIBOBJS LTLIBOBJS'
+ac_subst_vars='SHELL PATH_SEPARATOR PACKAGE_NAME PACKAGE_TARNAME PACKAGE_VERSION PACKAGE_STRING PACKAGE_BUGREPORT exec_prefix prefix program_transform_name bindir sbindir libexecdir datadir sysconfdir sharedstatedir localstatedir libdir includedir oldincludedir infodir mandir build_alias host_alias target_alias DEFS ECHO_C ECHO_N ECHO_T LIBS CC CFLAGS LDFLAGS CPPFLAGS ac_ct_CC EXEEXT OBJEXT INSTALL_PROGRAM INSTALL_SCRIPT INSTALL_DATA CYGPATH_W PACKAGE VERSION ACLOCAL AUTOCONF AUTOMAKE AUTOHEADER MAKEINFO install_sh STRIP ac_ct_STRIP INSTALL_STRIP_PROGRAM mkdir_p AWK SET_MAKE am__leading_dot AMTAR am__tar am__untar DEPDIR am__include am__quote AMDEP_TRUE AMDEP_FALSE AMDEPBACKSLASH CCDEPMODE am__fastdepCC_TRUE am__fastdepCC_FALSE build build_cpu build_vendor build_os host host_cpu host_vendor host_os SED EGREP LN_S ECHO AR ac_ct_AR RANLIB ac_ct_RANLIB CPP CXX CXXFLAGS ac_ct_CXX CXXDEPMODE am__fastdepCXX_TRUE am__fastdepCXX_FALSE CXXCPP F77 FFLAGS ac_ct_F77 LIBTOOL JNI_LDFLAGS JNI_CPPFLAGS LIBOBJS LTLIBOBJS'
ac_subst_files=''
# Initialize some variables set by options.
@@ -1024,6 +1024,7 @@ if test -n "$ac_init_help"; then
Optional Features:
--disable-FEATURE do not include FEATURE (same as --enable-FEATURE=no)
--enable-FEATURE[=ARG] include FEATURE [ARG=yes]
+ --disable-largefile omit support for large files
--disable-dependency-tracking speeds up one-time build
--enable-dependency-tracking do not reject slow dependency extractors
--enable-shared[=PKGS]
@@ -1523,6 +1524,1301 @@ ac_configure="$SHELL $ac_aux_dir/configure" # This should be Cygnus configure.
ac_config_headers="$ac_config_headers config.h"
+ac_ext=c
+ac_cpp='$CPP $CPPFLAGS'
+ac_compile='$CC -c $CFLAGS $CPPFLAGS conftest.$ac_ext >&5'
+ac_link='$CC -o conftest$ac_exeext $CFLAGS $CPPFLAGS $LDFLAGS conftest.$ac_ext $LIBS >&5'
+ac_compiler_gnu=$ac_cv_c_compiler_gnu
+if test -n "$ac_tool_prefix"; then
+ # Extract the first word of "${ac_tool_prefix}gcc", so it can be a program name with args.
+set dummy ${ac_tool_prefix}gcc; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$CC"; then
+ ac_cv_prog_CC="$CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_CC="${ac_tool_prefix}gcc"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+CC=$ac_cv_prog_CC
+if test -n "$CC"; then
+ echo "$as_me:$LINENO: result: $CC" >&5
+echo "${ECHO_T}$CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+fi
+if test -z "$ac_cv_prog_CC"; then
+ ac_ct_CC=$CC
+ # Extract the first word of "gcc", so it can be a program name with args.
+set dummy gcc; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_ac_ct_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$ac_ct_CC"; then
+ ac_cv_prog_ac_ct_CC="$ac_ct_CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_ac_ct_CC="gcc"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+ac_ct_CC=$ac_cv_prog_ac_ct_CC
+if test -n "$ac_ct_CC"; then
+ echo "$as_me:$LINENO: result: $ac_ct_CC" >&5
+echo "${ECHO_T}$ac_ct_CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+ CC=$ac_ct_CC
+else
+ CC="$ac_cv_prog_CC"
+fi
+
+if test -z "$CC"; then
+ if test -n "$ac_tool_prefix"; then
+ # Extract the first word of "${ac_tool_prefix}cc", so it can be a program name with args.
+set dummy ${ac_tool_prefix}cc; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$CC"; then
+ ac_cv_prog_CC="$CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_CC="${ac_tool_prefix}cc"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+CC=$ac_cv_prog_CC
+if test -n "$CC"; then
+ echo "$as_me:$LINENO: result: $CC" >&5
+echo "${ECHO_T}$CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+fi
+if test -z "$ac_cv_prog_CC"; then
+ ac_ct_CC=$CC
+ # Extract the first word of "cc", so it can be a program name with args.
+set dummy cc; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_ac_ct_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$ac_ct_CC"; then
+ ac_cv_prog_ac_ct_CC="$ac_ct_CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_ac_ct_CC="cc"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+ac_ct_CC=$ac_cv_prog_ac_ct_CC
+if test -n "$ac_ct_CC"; then
+ echo "$as_me:$LINENO: result: $ac_ct_CC" >&5
+echo "${ECHO_T}$ac_ct_CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+ CC=$ac_ct_CC
+else
+ CC="$ac_cv_prog_CC"
+fi
+
+fi
+if test -z "$CC"; then
+ # Extract the first word of "cc", so it can be a program name with args.
+set dummy cc; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$CC"; then
+ ac_cv_prog_CC="$CC" # Let the user override the test.
+else
+ ac_prog_rejected=no
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ if test "$as_dir/$ac_word$ac_exec_ext" = "/usr/ucb/cc"; then
+ ac_prog_rejected=yes
+ continue
+ fi
+ ac_cv_prog_CC="cc"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+if test $ac_prog_rejected = yes; then
+ # We found a bogon in the path, so make sure we never use it.
+ set dummy $ac_cv_prog_CC
+ shift
+ if test $# != 0; then
+ # We chose a different compiler from the bogus one.
+ # However, it has the same basename, so the bogon will be chosen
+ # first if we set CC to just the basename; use the full file name.
+ shift
+ ac_cv_prog_CC="$as_dir/$ac_word${1+' '}$@"
+ fi
+fi
+fi
+fi
+CC=$ac_cv_prog_CC
+if test -n "$CC"; then
+ echo "$as_me:$LINENO: result: $CC" >&5
+echo "${ECHO_T}$CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+fi
+if test -z "$CC"; then
+ if test -n "$ac_tool_prefix"; then
+ for ac_prog in cl
+ do
+ # Extract the first word of "$ac_tool_prefix$ac_prog", so it can be a program name with args.
+set dummy $ac_tool_prefix$ac_prog; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$CC"; then
+ ac_cv_prog_CC="$CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_CC="$ac_tool_prefix$ac_prog"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+CC=$ac_cv_prog_CC
+if test -n "$CC"; then
+ echo "$as_me:$LINENO: result: $CC" >&5
+echo "${ECHO_T}$CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+ test -n "$CC" && break
+ done
+fi
+if test -z "$CC"; then
+ ac_ct_CC=$CC
+ for ac_prog in cl
+do
+ # Extract the first word of "$ac_prog", so it can be a program name with args.
+set dummy $ac_prog; ac_word=$2
+echo "$as_me:$LINENO: checking for $ac_word" >&5
+echo $ECHO_N "checking for $ac_word... $ECHO_C" >&6
+if test "${ac_cv_prog_ac_ct_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -n "$ac_ct_CC"; then
+ ac_cv_prog_ac_ct_CC="$ac_ct_CC" # Let the user override the test.
+else
+as_save_IFS=$IFS; IFS=$PATH_SEPARATOR
+for as_dir in $PATH
+do
+ IFS=$as_save_IFS
+ test -z "$as_dir" && as_dir=.
+ for ac_exec_ext in '' $ac_executable_extensions; do
+ if $as_executable_p "$as_dir/$ac_word$ac_exec_ext"; then
+ ac_cv_prog_ac_ct_CC="$ac_prog"
+ echo "$as_me:$LINENO: found $as_dir/$ac_word$ac_exec_ext" >&5
+ break 2
+ fi
+done
+done
+
+fi
+fi
+ac_ct_CC=$ac_cv_prog_ac_ct_CC
+if test -n "$ac_ct_CC"; then
+ echo "$as_me:$LINENO: result: $ac_ct_CC" >&5
+echo "${ECHO_T}$ac_ct_CC" >&6
+else
+ echo "$as_me:$LINENO: result: no" >&5
+echo "${ECHO_T}no" >&6
+fi
+
+ test -n "$ac_ct_CC" && break
+done
+
+ CC=$ac_ct_CC
+fi
+
+fi
+
+
+test -z "$CC" && { { echo "$as_me:$LINENO: error: no acceptable C compiler found in \$PATH
+See \`config.log' for more details." >&5
+echo "$as_me: error: no acceptable C compiler found in \$PATH
+See \`config.log' for more details." >&2;}
+ { (exit 1); exit 1; }; }
+
+# Provide some information about the compiler.
+echo "$as_me:$LINENO:" \
+ "checking for C compiler version" >&5
+ac_compiler=`set X $ac_compile; echo $2`
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler --version &5\"") >&5
+ (eval $ac_compiler --version &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler -v &5\"") >&5
+ (eval $ac_compiler -v &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler -V &5\"") >&5
+ (eval $ac_compiler -V &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
+
+cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+ac_clean_files_save=$ac_clean_files
+ac_clean_files="$ac_clean_files a.out a.exe b.out"
+# Try to create an executable without -o first, disregard a.out.
+# It will help us diagnose broken compilers, and finding out an intuition
+# of exeext.
+echo "$as_me:$LINENO: checking for C compiler default output file name" >&5
+echo $ECHO_N "checking for C compiler default output file name... $ECHO_C" >&6
+ac_link_default=`echo "$ac_link" | sed 's/ -o *conftest[^ ]*//'`
+if { (eval echo "$as_me:$LINENO: \"$ac_link_default\"") >&5
+ (eval $ac_link_default) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; then
+ # Find the output, starting from the most likely. This scheme is
+# not robust to junk in `.', hence go to wildcards (a.*) only as a last
+# resort.
+
+# Be careful to initialize this variable, since it used to be cached.
+# Otherwise an old cache value of `no' led to `EXEEXT = no' in a Makefile.
+ac_cv_exeext=
+# b.out is created by i960 compilers.
+for ac_file in a_out.exe a.exe conftest.exe a.out conftest a.* conftest.* b.out
+do
+ test -f "$ac_file" || continue
+ case $ac_file in
+ *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg | *.o | *.obj )
+ ;;
+ conftest.$ac_ext )
+ # This is the source file.
+ ;;
+ [ab].out )
+ # We found the default executable, but exeext='' is most
+ # certainly right.
+ break;;
+ *.* )
+ ac_cv_exeext=`expr "$ac_file" : '[^.]*\(\..*\)'`
+ # FIXME: I believe we export ac_cv_exeext for Libtool,
+ # but it would be cool to find out if it's true. Does anybody
+ # maintain Libtool? --akim.
+ export ac_cv_exeext
+ break;;
+ * )
+ break;;
+ esac
+done
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+{ { echo "$as_me:$LINENO: error: C compiler cannot create executables
+See \`config.log' for more details." >&5
+echo "$as_me: error: C compiler cannot create executables
+See \`config.log' for more details." >&2;}
+ { (exit 77); exit 77; }; }
+fi
+
+ac_exeext=$ac_cv_exeext
+echo "$as_me:$LINENO: result: $ac_file" >&5
+echo "${ECHO_T}$ac_file" >&6
+
+# Check the compiler produces executables we can run. If not, either
+# the compiler is broken, or we cross compile.
+echo "$as_me:$LINENO: checking whether the C compiler works" >&5
+echo $ECHO_N "checking whether the C compiler works... $ECHO_C" >&6
+# FIXME: These cross compiler hacks should be removed for Autoconf 3.0
+# If not cross compiling, check that we can run a simple program.
+if test "$cross_compiling" != yes; then
+ if { ac_try='./$ac_file'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ cross_compiling=no
+ else
+ if test "$cross_compiling" = maybe; then
+ cross_compiling=yes
+ else
+ { { echo "$as_me:$LINENO: error: cannot run C compiled programs.
+If you meant to cross compile, use \`--host'.
+See \`config.log' for more details." >&5
+echo "$as_me: error: cannot run C compiled programs.
+If you meant to cross compile, use \`--host'.
+See \`config.log' for more details." >&2;}
+ { (exit 1); exit 1; }; }
+ fi
+ fi
+fi
+echo "$as_me:$LINENO: result: yes" >&5
+echo "${ECHO_T}yes" >&6
+
+rm -f a.out a.exe conftest$ac_cv_exeext b.out
+ac_clean_files=$ac_clean_files_save
+# Check the compiler produces executables we can run. If not, either
+# the compiler is broken, or we cross compile.
+echo "$as_me:$LINENO: checking whether we are cross compiling" >&5
+echo $ECHO_N "checking whether we are cross compiling... $ECHO_C" >&6
+echo "$as_me:$LINENO: result: $cross_compiling" >&5
+echo "${ECHO_T}$cross_compiling" >&6
+
+echo "$as_me:$LINENO: checking for suffix of executables" >&5
+echo $ECHO_N "checking for suffix of executables... $ECHO_C" >&6
+if { (eval echo "$as_me:$LINENO: \"$ac_link\"") >&5
+ (eval $ac_link) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; then
+ # If both `conftest.exe' and `conftest' are `present' (well, observable)
+# catch `conftest.exe'. For instance with Cygwin, `ls conftest' will
+# work properly (i.e., refer to `conftest.exe'), while it won't with
+# `rm'.
+for ac_file in conftest.exe conftest conftest.*; do
+ test -f "$ac_file" || continue
+ case $ac_file in
+ *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg | *.o | *.obj ) ;;
+ *.* ) ac_cv_exeext=`expr "$ac_file" : '[^.]*\(\..*\)'`
+ export ac_cv_exeext
+ break;;
+ * ) break;;
+ esac
+done
+else
+ { { echo "$as_me:$LINENO: error: cannot compute suffix of executables: cannot compile and link
+See \`config.log' for more details." >&5
+echo "$as_me: error: cannot compute suffix of executables: cannot compile and link
+See \`config.log' for more details." >&2;}
+ { (exit 1); exit 1; }; }
+fi
+
+rm -f conftest$ac_cv_exeext
+echo "$as_me:$LINENO: result: $ac_cv_exeext" >&5
+echo "${ECHO_T}$ac_cv_exeext" >&6
+
+rm -f conftest.$ac_ext
+EXEEXT=$ac_cv_exeext
+ac_exeext=$EXEEXT
+echo "$as_me:$LINENO: checking for suffix of object files" >&5
+echo $ECHO_N "checking for suffix of object files... $ECHO_C" >&6
+if test "${ac_cv_objext+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.o conftest.obj
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; then
+ for ac_file in `(ls conftest.o conftest.obj; ls conftest.*) 2>/dev/null`; do
+ case $ac_file in
+ *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg ) ;;
+ *) ac_cv_objext=`expr "$ac_file" : '.*\.\(.*\)'`
+ break;;
+ esac
+done
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+{ { echo "$as_me:$LINENO: error: cannot compute suffix of object files: cannot compile
+See \`config.log' for more details." >&5
+echo "$as_me: error: cannot compute suffix of object files: cannot compile
+See \`config.log' for more details." >&2;}
+ { (exit 1); exit 1; }; }
+fi
+
+rm -f conftest.$ac_cv_objext conftest.$ac_ext
+fi
+echo "$as_me:$LINENO: result: $ac_cv_objext" >&5
+echo "${ECHO_T}$ac_cv_objext" >&6
+OBJEXT=$ac_cv_objext
+ac_objext=$OBJEXT
+echo "$as_me:$LINENO: checking whether we are using the GNU C compiler" >&5
+echo $ECHO_N "checking whether we are using the GNU C compiler... $ECHO_C" >&6
+if test "${ac_cv_c_compiler_gnu+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+
+int
+main ()
+{
+#ifndef __GNUC__
+ choke me
+#endif
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_compiler_gnu=yes
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+ac_compiler_gnu=no
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ac_cv_c_compiler_gnu=$ac_compiler_gnu
+
+fi
+echo "$as_me:$LINENO: result: $ac_cv_c_compiler_gnu" >&5
+echo "${ECHO_T}$ac_cv_c_compiler_gnu" >&6
+GCC=`test $ac_compiler_gnu = yes && echo yes`
+ac_test_CFLAGS=${CFLAGS+set}
+ac_save_CFLAGS=$CFLAGS
+CFLAGS="-g"
+echo "$as_me:$LINENO: checking whether $CC accepts -g" >&5
+echo $ECHO_N "checking whether $CC accepts -g... $ECHO_C" >&6
+if test "${ac_cv_prog_cc_g+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_prog_cc_g=yes
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+ac_cv_prog_cc_g=no
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+fi
+echo "$as_me:$LINENO: result: $ac_cv_prog_cc_g" >&5
+echo "${ECHO_T}$ac_cv_prog_cc_g" >&6
+if test "$ac_test_CFLAGS" = set; then
+ CFLAGS=$ac_save_CFLAGS
+elif test $ac_cv_prog_cc_g = yes; then
+ if test "$GCC" = yes; then
+ CFLAGS="-g -O2"
+ else
+ CFLAGS="-g"
+ fi
+else
+ if test "$GCC" = yes; then
+ CFLAGS="-O2"
+ else
+ CFLAGS=
+ fi
+fi
+echo "$as_me:$LINENO: checking for $CC option to accept ANSI C" >&5
+echo $ECHO_N "checking for $CC option to accept ANSI C... $ECHO_C" >&6
+if test "${ac_cv_prog_cc_stdc+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ ac_cv_prog_cc_stdc=no
+ac_save_CC=$CC
+cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#include
+#include
+#include
+#include
+/* Most of the following tests are stolen from RCS 5.7's src/conf.sh. */
+struct buf { int x; };
+FILE * (*rcsopen) (struct buf *, struct stat *, int);
+static char *e (p, i)
+ char **p;
+ int i;
+{
+ return p[i];
+}
+static char *f (char * (*g) (char **, int), char **p, ...)
+{
+ char *s;
+ va_list v;
+ va_start (v,p);
+ s = g (p, va_arg (v,int));
+ va_end (v);
+ return s;
+}
+
+/* OSF 4.0 Compaq cc is some sort of almost-ANSI by default. It has
+ function prototypes and stuff, but not '\xHH' hex character constants.
+ These don't provoke an error unfortunately, instead are silently treated
+ as 'x'. The following induces an error, until -std1 is added to get
+ proper ANSI mode. Curiously '\x00'!='x' always comes out true, for an
+ array size at least. It's necessary to write '\x00'==0 to get something
+ that's true only with -std1. */
+int osf4_cc_array ['\x00' == 0 ? 1 : -1];
+
+int test (int i, double x);
+struct s1 {int (*f) (int a);};
+struct s2 {int (*f) (double a);};
+int pairnames (int, char **, FILE *(*)(struct buf *, struct stat *, int), int, int);
+int argc;
+char **argv;
+int
+main ()
+{
+return f (e, argv, 0) != argv[0] || f (e, argv, 1) != argv[1];
+ ;
+ return 0;
+}
+_ACEOF
+# Don't try gcc -ansi; that turns off useful extensions and
+# breaks some systems' header files.
+# AIX -qlanglvl=ansi
+# Ultrix and OSF/1 -std1
+# HP-UX 10.20 and later -Ae
+# HP-UX older versions -Aa -D_HPUX_SOURCE
+# SVR4 -Xc -D__EXTENSIONS__
+for ac_arg in "" -qlanglvl=ansi -std1 -Ae "-Aa -D_HPUX_SOURCE" "-Xc -D__EXTENSIONS__"
+do
+ CC="$ac_save_CC $ac_arg"
+ rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_prog_cc_stdc=$ac_arg
+break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext
+done
+rm -f conftest.$ac_ext conftest.$ac_objext
+CC=$ac_save_CC
+
+fi
+
+case "x$ac_cv_prog_cc_stdc" in
+ x|xno)
+ echo "$as_me:$LINENO: result: none needed" >&5
+echo "${ECHO_T}none needed" >&6 ;;
+ *)
+ echo "$as_me:$LINENO: result: $ac_cv_prog_cc_stdc" >&5
+echo "${ECHO_T}$ac_cv_prog_cc_stdc" >&6
+ CC="$CC $ac_cv_prog_cc_stdc" ;;
+esac
+
+# Some people use a C++ compiler to compile C. Since we use `exit',
+# in C++ we need to declare it. In case someone uses the same compiler
+# for both compiling C and C++ we need to have the C++ compiler decide
+# the declaration of exit, since it's the most demanding environment.
+cat >conftest.$ac_ext <<_ACEOF
+#ifndef __cplusplus
+ choke me
+#endif
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ for ac_declaration in \
+ '' \
+ 'extern "C" void std::exit (int) throw (); using std::exit;' \
+ 'extern "C" void std::exit (int); using std::exit;' \
+ 'extern "C" void exit (int) throw ();' \
+ 'extern "C" void exit (int);' \
+ 'void exit (int);'
+do
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_declaration
+#include
+int
+main ()
+{
+exit (42);
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ :
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+continue
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_declaration
+int
+main ()
+{
+exit (42);
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+done
+rm -f conftest*
+if test -n "$ac_declaration"; then
+ echo '#ifdef __cplusplus' >>confdefs.h
+ echo $ac_declaration >>confdefs.h
+ echo '#endif' >>confdefs.h
+fi
+
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ac_ext=c
+ac_cpp='$CPP $CPPFLAGS'
+ac_compile='$CC -c $CFLAGS $CPPFLAGS conftest.$ac_ext >&5'
+ac_link='$CC -o conftest$ac_exeext $CFLAGS $CPPFLAGS $LDFLAGS conftest.$ac_ext $LIBS >&5'
+ac_compiler_gnu=$ac_cv_c_compiler_gnu
+
+
+# Check whether --enable-largefile or --disable-largefile was given.
+if test "${enable_largefile+set}" = set; then
+ enableval="$enable_largefile"
+
+fi;
+if test "$enable_largefile" != no; then
+
+ echo "$as_me:$LINENO: checking for special C compiler options needed for large files" >&5
+echo $ECHO_N "checking for special C compiler options needed for large files... $ECHO_C" >&6
+if test "${ac_cv_sys_largefile_CC+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ ac_cv_sys_largefile_CC=no
+ if test "$GCC" != yes; then
+ ac_save_CC=$CC
+ while :; do
+ # IRIX 6.2 and later do not support large files by default,
+ # so use the C compiler's -n32 option if that helps.
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#include
+ /* Check that off_t can represent 2**63 - 1 correctly.
+ We can't simply define LARGE_OFF_T to be 9223372036854775807,
+ since some C++ compilers masquerading as C compilers
+ incorrectly reject 9223372036854775807. */
+#define LARGE_OFF_T (((off_t) 1 << 62) - 1 + ((off_t) 1 << 62))
+ int off_t_is_large[(LARGE_OFF_T % 2147483629 == 721
+ && LARGE_OFF_T % 2147483647 == 1)
+ ? 1 : -1];
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+ rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext
+ CC="$CC -n32"
+ rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_sys_largefile_CC=' -n32'; break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext
+ break
+ done
+ CC=$ac_save_CC
+ rm -f conftest.$ac_ext
+ fi
+fi
+echo "$as_me:$LINENO: result: $ac_cv_sys_largefile_CC" >&5
+echo "${ECHO_T}$ac_cv_sys_largefile_CC" >&6
+ if test "$ac_cv_sys_largefile_CC" != no; then
+ CC=$CC$ac_cv_sys_largefile_CC
+ fi
+
+ echo "$as_me:$LINENO: checking for _FILE_OFFSET_BITS value needed for large files" >&5
+echo $ECHO_N "checking for _FILE_OFFSET_BITS value needed for large files... $ECHO_C" >&6
+if test "${ac_cv_sys_file_offset_bits+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ while :; do
+ ac_cv_sys_file_offset_bits=no
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#include
+ /* Check that off_t can represent 2**63 - 1 correctly.
+ We can't simply define LARGE_OFF_T to be 9223372036854775807,
+ since some C++ compilers masquerading as C compilers
+ incorrectly reject 9223372036854775807. */
+#define LARGE_OFF_T (((off_t) 1 << 62) - 1 + ((off_t) 1 << 62))
+ int off_t_is_large[(LARGE_OFF_T % 2147483629 == 721
+ && LARGE_OFF_T % 2147483647 == 1)
+ ? 1 : -1];
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#define _FILE_OFFSET_BITS 64
+#include
+ /* Check that off_t can represent 2**63 - 1 correctly.
+ We can't simply define LARGE_OFF_T to be 9223372036854775807,
+ since some C++ compilers masquerading as C compilers
+ incorrectly reject 9223372036854775807. */
+#define LARGE_OFF_T (((off_t) 1 << 62) - 1 + ((off_t) 1 << 62))
+ int off_t_is_large[(LARGE_OFF_T % 2147483629 == 721
+ && LARGE_OFF_T % 2147483647 == 1)
+ ? 1 : -1];
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_sys_file_offset_bits=64; break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ break
+done
+fi
+echo "$as_me:$LINENO: result: $ac_cv_sys_file_offset_bits" >&5
+echo "${ECHO_T}$ac_cv_sys_file_offset_bits" >&6
+if test "$ac_cv_sys_file_offset_bits" != no; then
+
+cat >>confdefs.h <<_ACEOF
+#define _FILE_OFFSET_BITS $ac_cv_sys_file_offset_bits
+_ACEOF
+
+fi
+rm -f conftest*
+ echo "$as_me:$LINENO: checking for _LARGE_FILES value needed for large files" >&5
+echo $ECHO_N "checking for _LARGE_FILES value needed for large files... $ECHO_C" >&6
+if test "${ac_cv_sys_large_files+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ while :; do
+ ac_cv_sys_large_files=no
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#include
+ /* Check that off_t can represent 2**63 - 1 correctly.
+ We can't simply define LARGE_OFF_T to be 9223372036854775807,
+ since some C++ compilers masquerading as C compilers
+ incorrectly reject 9223372036854775807. */
+#define LARGE_OFF_T (((off_t) 1 << 62) - 1 + ((off_t) 1 << 62))
+ int off_t_is_large[(LARGE_OFF_T % 2147483629 == 721
+ && LARGE_OFF_T % 2147483647 == 1)
+ ? 1 : -1];
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#define _LARGE_FILES 1
+#include
+ /* Check that off_t can represent 2**63 - 1 correctly.
+ We can't simply define LARGE_OFF_T to be 9223372036854775807,
+ since some C++ compilers masquerading as C compilers
+ incorrectly reject 9223372036854775807. */
+#define LARGE_OFF_T (((off_t) 1 << 62) - 1 + ((off_t) 1 << 62))
+ int off_t_is_large[(LARGE_OFF_T % 2147483629 == 721
+ && LARGE_OFF_T % 2147483647 == 1)
+ ? 1 : -1];
+int
+main ()
+{
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_sys_large_files=1; break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ break
+done
+fi
+echo "$as_me:$LINENO: result: $ac_cv_sys_large_files" >&5
+echo "${ECHO_T}$ac_cv_sys_large_files" >&6
+if test "$ac_cv_sys_large_files" != no; then
+
+cat >>confdefs.h <<_ACEOF
+#define _LARGE_FILES $ac_cv_sys_large_files
+_ACEOF
+
+fi
+rm -f conftest*
+fi
+
am__api_version="1.9"
# Find a good install program. We prefer a C program (faster),
@@ -1786,6 +3082,71 @@ else
fi
rmdir .tst 2>/dev/null
+DEPDIR="${am__leading_dot}deps"
+
+ ac_config_commands="$ac_config_commands depfiles"
+
+
+am_make=${MAKE-make}
+cat > confinc << 'END'
+am__doit:
+ @echo done
+.PHONY: am__doit
+END
+# If we don't find an include directive, just comment out the code.
+echo "$as_me:$LINENO: checking for style of include used by $am_make" >&5
+echo $ECHO_N "checking for style of include used by $am_make... $ECHO_C" >&6
+am__include="#"
+am__quote=
+_am_result=none
+# First try GNU make style include.
+echo "include confinc" > confmf
+# We grep out `Entering directory' and `Leaving directory'
+# messages which can occur if `w' ends up in MAKEFLAGS.
+# In particular we don't look at `^make:' because GNU make might
+# be invoked under some other name (usually "gmake"), in which
+# case it prints its new name instead of `make'.
+if test "`$am_make -s -f confmf 2> /dev/null | grep -v 'ing directory'`" = "done"; then
+ am__include=include
+ am__quote=
+ _am_result=GNU
+fi
+# Now try BSD make style include.
+if test "$am__include" = "#"; then
+ echo '.include "confinc"' > confmf
+ if test "`$am_make -s -f confmf 2> /dev/null`" = "done"; then
+ am__include=.include
+ am__quote="\""
+ _am_result=BSD
+ fi
+fi
+
+
+echo "$as_me:$LINENO: result: $_am_result" >&5
+echo "${ECHO_T}$_am_result" >&6
+rm -f confinc confmf
+
+# Check whether --enable-dependency-tracking or --disable-dependency-tracking was given.
+if test "${enable_dependency_tracking+set}" = set; then
+ enableval="$enable_dependency_tracking"
+
+fi;
+if test "x$enable_dependency_tracking" != xno; then
+ am_depcomp="$ac_aux_dir/depcomp"
+ AMDEPBACKSLASH='\'
+fi
+
+
+if test "x$enable_dependency_tracking" != xno; then
+ AMDEP_TRUE=
+ AMDEP_FALSE='#'
+else
+ AMDEP_TRUE='#'
+ AMDEP_FALSE=
+fi
+
+
+
# test to see if srcdir already configured
if test "`cd $srcdir && pwd`" != "`pwd`" &&
test -f $srcdir/config.status; then
@@ -1916,23 +3277,132 @@ else
echo "${ECHO_T}no" >&6
fi
- STRIP=$ac_ct_STRIP
+ STRIP=$ac_ct_STRIP
+else
+ STRIP="$ac_cv_prog_STRIP"
+fi
+
+fi
+INSTALL_STRIP_PROGRAM="\${SHELL} \$(install_sh) -c -s"
+
+# We need awk for the "check" target. The system "awk" is bad on
+# some platforms.
+# Always define AMTAR for backward compatibility.
+
+AMTAR=${AMTAR-"${am_missing_run}tar"}
+
+am__tar='${AMTAR} chof - "$$tardir"'; am__untar='${AMTAR} xf -'
+
+
+
+
+depcc="$CC" am_compiler_list=
+
+echo "$as_me:$LINENO: checking dependency style of $depcc" >&5
+echo $ECHO_N "checking dependency style of $depcc... $ECHO_C" >&6
+if test "${am_cv_CC_dependencies_compiler_type+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ if test -z "$AMDEP_TRUE" && test -f "$am_depcomp"; then
+ # We make a subdir and do the tests there. Otherwise we can end up
+ # making bogus files that we don't know about and never remove. For
+ # instance it was reported that on HP-UX the gcc test will end up
+ # making a dummy file named `D' -- because `-MD' means `put the output
+ # in D'.
+ mkdir conftest.dir
+ # Copy depcomp to subdir because otherwise we won't find it if we're
+ # using a relative directory.
+ cp "$am_depcomp" conftest.dir
+ cd conftest.dir
+ # We will build objects and dependencies in a subdirectory because
+ # it helps to detect inapplicable dependency modes. For instance
+ # both Tru64's cc and ICC support -MD to output dependencies as a
+ # side effect of compilation, but ICC will put the dependencies in
+ # the current directory while Tru64 will put them in the object
+ # directory.
+ mkdir sub
+
+ am_cv_CC_dependencies_compiler_type=none
+ if test "$am_compiler_list" = ""; then
+ am_compiler_list=`sed -n 's/^#*\([a-zA-Z0-9]*\))$/\1/p' < ./depcomp`
+ fi
+ for depmode in $am_compiler_list; do
+ # Setup a source with many dependencies, because some compilers
+ # like to wrap large dependency lists on column 80 (with \), and
+ # we should not choose a depcomp mode which is confused by this.
+ #
+ # We need to recreate these files for each test, as the compiler may
+ # overwrite some of them when testing with obscure command lines.
+ # This happens at least with the AIX C compiler.
+ : > sub/conftest.c
+ for i in 1 2 3 4 5 6; do
+ echo '#include "conftst'$i'.h"' >> sub/conftest.c
+ # Using `: > sub/conftst$i.h' creates only sub/conftst1.h with
+ # Solaris 8's {/usr,}/bin/sh.
+ touch sub/conftst$i.h
+ done
+ echo "${am__include} ${am__quote}sub/conftest.Po${am__quote}" > confmf
+
+ case $depmode in
+ nosideeffect)
+ # after this tag, mechanisms are not by side-effect, so they'll
+ # only be used when explicitly requested
+ if test "x$enable_dependency_tracking" = xyes; then
+ continue
+ else
+ break
+ fi
+ ;;
+ none) break ;;
+ esac
+ # We check with `-c' and `-o' for the sake of the "dashmstdout"
+ # mode. It turns out that the SunPro C++ compiler does not properly
+ # handle `-M -o', and we need to detect this.
+ if depmode=$depmode \
+ source=sub/conftest.c object=sub/conftest.${OBJEXT-o} \
+ depfile=sub/conftest.Po tmpdepfile=sub/conftest.TPo \
+ $SHELL ./depcomp $depcc -c -o sub/conftest.${OBJEXT-o} sub/conftest.c \
+ >/dev/null 2>conftest.err &&
+ grep sub/conftst6.h sub/conftest.Po > /dev/null 2>&1 &&
+ grep sub/conftest.${OBJEXT-o} sub/conftest.Po > /dev/null 2>&1 &&
+ ${MAKE-make} -s -f confmf > /dev/null 2>&1; then
+ # icc doesn't choke on unknown options, it will just issue warnings
+ # or remarks (even with -Werror). So we grep stderr for any message
+ # that says an option was ignored or not supported.
+ # When given -MP, icc 7.0 and 7.1 complain thusly:
+ # icc: Command line warning: ignoring option '-M'; no argument required
+ # The diagnosis changed in icc 8.0:
+ # icc: Command line remark: option '-MP' not supported
+ if (grep 'ignoring option' conftest.err ||
+ grep 'not supported' conftest.err) >/dev/null 2>&1; then :; else
+ am_cv_CC_dependencies_compiler_type=$depmode
+ break
+ fi
+ fi
+ done
+
+ cd ..
+ rm -rf conftest.dir
else
- STRIP="$ac_cv_prog_STRIP"
+ am_cv_CC_dependencies_compiler_type=none
fi
fi
-INSTALL_STRIP_PROGRAM="\${SHELL} \$(install_sh) -c -s"
-
-# We need awk for the "check" target. The system "awk" is bad on
-# some platforms.
-# Always define AMTAR for backward compatibility.
-
-AMTAR=${AMTAR-"${am_missing_run}tar"}
+echo "$as_me:$LINENO: result: $am_cv_CC_dependencies_compiler_type" >&5
+echo "${ECHO_T}$am_cv_CC_dependencies_compiler_type" >&6
+CCDEPMODE=depmode=$am_cv_CC_dependencies_compiler_type
-am__tar='${AMTAR} chof - "$$tardir"'; am__untar='${AMTAR} xf -'
+if
+ test "x$enable_dependency_tracking" != xno \
+ && test "$am_cv_CC_dependencies_compiler_type" = gcc3; then
+ am__fastdepCC_TRUE=
+ am__fastdepCC_FALSE='#'
+else
+ am__fastdepCC_TRUE='#'
+ am__fastdepCC_FALSE=
+fi
@@ -2255,225 +3725,24 @@ See \`config.log' for more details." >&2;}
# Provide some information about the compiler.
echo "$as_me:$LINENO:" \
- "checking for C compiler version" >&5
-ac_compiler=`set X $ac_compile; echo $2`
-{ (eval echo "$as_me:$LINENO: \"$ac_compiler --version &5\"") >&5
- (eval $ac_compiler --version &5) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }
-{ (eval echo "$as_me:$LINENO: \"$ac_compiler -v &5\"") >&5
- (eval $ac_compiler -v &5) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }
-{ (eval echo "$as_me:$LINENO: \"$ac_compiler -V &5\"") >&5
- (eval $ac_compiler -V &5) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }
-
-cat >conftest.$ac_ext <<_ACEOF
-/* confdefs.h. */
-_ACEOF
-cat confdefs.h >>conftest.$ac_ext
-cat >>conftest.$ac_ext <<_ACEOF
-/* end confdefs.h. */
-
-int
-main ()
-{
-
- ;
- return 0;
-}
-_ACEOF
-ac_clean_files_save=$ac_clean_files
-ac_clean_files="$ac_clean_files a.out a.exe b.out"
-# Try to create an executable without -o first, disregard a.out.
-# It will help us diagnose broken compilers, and finding out an intuition
-# of exeext.
-echo "$as_me:$LINENO: checking for C compiler default output file name" >&5
-echo $ECHO_N "checking for C compiler default output file name... $ECHO_C" >&6
-ac_link_default=`echo "$ac_link" | sed 's/ -o *conftest[^ ]*//'`
-if { (eval echo "$as_me:$LINENO: \"$ac_link_default\"") >&5
- (eval $ac_link_default) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; then
- # Find the output, starting from the most likely. This scheme is
-# not robust to junk in `.', hence go to wildcards (a.*) only as a last
-# resort.
-
-# Be careful to initialize this variable, since it used to be cached.
-# Otherwise an old cache value of `no' led to `EXEEXT = no' in a Makefile.
-ac_cv_exeext=
-# b.out is created by i960 compilers.
-for ac_file in a_out.exe a.exe conftest.exe a.out conftest a.* conftest.* b.out
-do
- test -f "$ac_file" || continue
- case $ac_file in
- *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg | *.o | *.obj )
- ;;
- conftest.$ac_ext )
- # This is the source file.
- ;;
- [ab].out )
- # We found the default executable, but exeext='' is most
- # certainly right.
- break;;
- *.* )
- ac_cv_exeext=`expr "$ac_file" : '[^.]*\(\..*\)'`
- # FIXME: I believe we export ac_cv_exeext for Libtool,
- # but it would be cool to find out if it's true. Does anybody
- # maintain Libtool? --akim.
- export ac_cv_exeext
- break;;
- * )
- break;;
- esac
-done
-else
- echo "$as_me: failed program was:" >&5
-sed 's/^/| /' conftest.$ac_ext >&5
-
-{ { echo "$as_me:$LINENO: error: C compiler cannot create executables
-See \`config.log' for more details." >&5
-echo "$as_me: error: C compiler cannot create executables
-See \`config.log' for more details." >&2;}
- { (exit 77); exit 77; }; }
-fi
-
-ac_exeext=$ac_cv_exeext
-echo "$as_me:$LINENO: result: $ac_file" >&5
-echo "${ECHO_T}$ac_file" >&6
-
-# Check the compiler produces executables we can run. If not, either
-# the compiler is broken, or we cross compile.
-echo "$as_me:$LINENO: checking whether the C compiler works" >&5
-echo $ECHO_N "checking whether the C compiler works... $ECHO_C" >&6
-# FIXME: These cross compiler hacks should be removed for Autoconf 3.0
-# If not cross compiling, check that we can run a simple program.
-if test "$cross_compiling" != yes; then
- if { ac_try='./$ac_file'
- { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
- (eval $ac_try) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; }; then
- cross_compiling=no
- else
- if test "$cross_compiling" = maybe; then
- cross_compiling=yes
- else
- { { echo "$as_me:$LINENO: error: cannot run C compiled programs.
-If you meant to cross compile, use \`--host'.
-See \`config.log' for more details." >&5
-echo "$as_me: error: cannot run C compiled programs.
-If you meant to cross compile, use \`--host'.
-See \`config.log' for more details." >&2;}
- { (exit 1); exit 1; }; }
- fi
- fi
-fi
-echo "$as_me:$LINENO: result: yes" >&5
-echo "${ECHO_T}yes" >&6
-
-rm -f a.out a.exe conftest$ac_cv_exeext b.out
-ac_clean_files=$ac_clean_files_save
-# Check the compiler produces executables we can run. If not, either
-# the compiler is broken, or we cross compile.
-echo "$as_me:$LINENO: checking whether we are cross compiling" >&5
-echo $ECHO_N "checking whether we are cross compiling... $ECHO_C" >&6
-echo "$as_me:$LINENO: result: $cross_compiling" >&5
-echo "${ECHO_T}$cross_compiling" >&6
-
-echo "$as_me:$LINENO: checking for suffix of executables" >&5
-echo $ECHO_N "checking for suffix of executables... $ECHO_C" >&6
-if { (eval echo "$as_me:$LINENO: \"$ac_link\"") >&5
- (eval $ac_link) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; then
- # If both `conftest.exe' and `conftest' are `present' (well, observable)
-# catch `conftest.exe'. For instance with Cygwin, `ls conftest' will
-# work properly (i.e., refer to `conftest.exe'), while it won't with
-# `rm'.
-for ac_file in conftest.exe conftest conftest.*; do
- test -f "$ac_file" || continue
- case $ac_file in
- *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg | *.o | *.obj ) ;;
- *.* ) ac_cv_exeext=`expr "$ac_file" : '[^.]*\(\..*\)'`
- export ac_cv_exeext
- break;;
- * ) break;;
- esac
-done
-else
- { { echo "$as_me:$LINENO: error: cannot compute suffix of executables: cannot compile and link
-See \`config.log' for more details." >&5
-echo "$as_me: error: cannot compute suffix of executables: cannot compile and link
-See \`config.log' for more details." >&2;}
- { (exit 1); exit 1; }; }
-fi
-
-rm -f conftest$ac_cv_exeext
-echo "$as_me:$LINENO: result: $ac_cv_exeext" >&5
-echo "${ECHO_T}$ac_cv_exeext" >&6
-
-rm -f conftest.$ac_ext
-EXEEXT=$ac_cv_exeext
-ac_exeext=$EXEEXT
-echo "$as_me:$LINENO: checking for suffix of object files" >&5
-echo $ECHO_N "checking for suffix of object files... $ECHO_C" >&6
-if test "${ac_cv_objext+set}" = set; then
- echo $ECHO_N "(cached) $ECHO_C" >&6
-else
- cat >conftest.$ac_ext <<_ACEOF
-/* confdefs.h. */
-_ACEOF
-cat confdefs.h >>conftest.$ac_ext
-cat >>conftest.$ac_ext <<_ACEOF
-/* end confdefs.h. */
-
-int
-main ()
-{
-
- ;
- return 0;
-}
-_ACEOF
-rm -f conftest.o conftest.obj
-if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
- (eval $ac_compile) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; then
- for ac_file in `(ls conftest.o conftest.obj; ls conftest.*) 2>/dev/null`; do
- case $ac_file in
- *.$ac_ext | *.xcoff | *.tds | *.d | *.pdb | *.xSYM | *.bb | *.bbg ) ;;
- *) ac_cv_objext=`expr "$ac_file" : '.*\.\(.*\)'`
- break;;
- esac
-done
-else
- echo "$as_me: failed program was:" >&5
-sed 's/^/| /' conftest.$ac_ext >&5
-
-{ { echo "$as_me:$LINENO: error: cannot compute suffix of object files: cannot compile
-See \`config.log' for more details." >&5
-echo "$as_me: error: cannot compute suffix of object files: cannot compile
-See \`config.log' for more details." >&2;}
- { (exit 1); exit 1; }; }
-fi
+ "checking for C compiler version" >&5
+ac_compiler=`set X $ac_compile; echo $2`
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler --version &5\"") >&5
+ (eval $ac_compiler --version &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler -v &5\"") >&5
+ (eval $ac_compiler -v &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
+{ (eval echo "$as_me:$LINENO: \"$ac_compiler -V &5\"") >&5
+ (eval $ac_compiler -V &5) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }
-rm -f conftest.$ac_cv_objext conftest.$ac_ext
-fi
-echo "$as_me:$LINENO: result: $ac_cv_objext" >&5
-echo "${ECHO_T}$ac_cv_objext" >&6
-OBJEXT=$ac_cv_objext
-ac_objext=$OBJEXT
echo "$as_me:$LINENO: checking whether we are using the GNU C compiler" >&5
echo $ECHO_N "checking whether we are using the GNU C compiler... $ECHO_C" >&6
if test "${ac_cv_c_compiler_gnu+set}" = set; then
@@ -2787,260 +4056,86 @@ if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
|| test ! -s conftest.err'
{ (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
(eval $ac_try) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; } &&
- { ac_try='test -s conftest.$ac_objext'
- { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
- (eval $ac_try) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; }; then
- :
-else
- echo "$as_me: failed program was:" >&5
-sed 's/^/| /' conftest.$ac_ext >&5
-
-continue
-fi
-rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
- cat >conftest.$ac_ext <<_ACEOF
-/* confdefs.h. */
-_ACEOF
-cat confdefs.h >>conftest.$ac_ext
-cat >>conftest.$ac_ext <<_ACEOF
-/* end confdefs.h. */
-$ac_declaration
-int
-main ()
-{
-exit (42);
- ;
- return 0;
-}
-_ACEOF
-rm -f conftest.$ac_objext
-if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
- (eval $ac_compile) 2>conftest.er1
- ac_status=$?
- grep -v '^ *+' conftest.er1 >conftest.err
- rm -f conftest.er1
- cat conftest.err >&5
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); } &&
- { ac_try='test -z "$ac_c_werror_flag"
- || test ! -s conftest.err'
- { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
- (eval $ac_try) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; } &&
- { ac_try='test -s conftest.$ac_objext'
- { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
- (eval $ac_try) 2>&5
- ac_status=$?
- echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); }; }; then
- break
-else
- echo "$as_me: failed program was:" >&5
-sed 's/^/| /' conftest.$ac_ext >&5
-
-fi
-rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
-done
-rm -f conftest*
-if test -n "$ac_declaration"; then
- echo '#ifdef __cplusplus' >>confdefs.h
- echo $ac_declaration >>confdefs.h
- echo '#endif' >>confdefs.h
-fi
-
-else
- echo "$as_me: failed program was:" >&5
-sed 's/^/| /' conftest.$ac_ext >&5
-
-fi
-rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
-ac_ext=c
-ac_cpp='$CPP $CPPFLAGS'
-ac_compile='$CC -c $CFLAGS $CPPFLAGS conftest.$ac_ext >&5'
-ac_link='$CC -o conftest$ac_exeext $CFLAGS $CPPFLAGS $LDFLAGS conftest.$ac_ext $LIBS >&5'
-ac_compiler_gnu=$ac_cv_c_compiler_gnu
-DEPDIR="${am__leading_dot}deps"
-
- ac_config_commands="$ac_config_commands depfiles"
-
-
-am_make=${MAKE-make}
-cat > confinc << 'END'
-am__doit:
- @echo done
-.PHONY: am__doit
-END
-# If we don't find an include directive, just comment out the code.
-echo "$as_me:$LINENO: checking for style of include used by $am_make" >&5
-echo $ECHO_N "checking for style of include used by $am_make... $ECHO_C" >&6
-am__include="#"
-am__quote=
-_am_result=none
-# First try GNU make style include.
-echo "include confinc" > confmf
-# We grep out `Entering directory' and `Leaving directory'
-# messages which can occur if `w' ends up in MAKEFLAGS.
-# In particular we don't look at `^make:' because GNU make might
-# be invoked under some other name (usually "gmake"), in which
-# case it prints its new name instead of `make'.
-if test "`$am_make -s -f confmf 2> /dev/null | grep -v 'ing directory'`" = "done"; then
- am__include=include
- am__quote=
- _am_result=GNU
-fi
-# Now try BSD make style include.
-if test "$am__include" = "#"; then
- echo '.include "confinc"' > confmf
- if test "`$am_make -s -f confmf 2> /dev/null`" = "done"; then
- am__include=.include
- am__quote="\""
- _am_result=BSD
- fi
-fi
-
-
-echo "$as_me:$LINENO: result: $_am_result" >&5
-echo "${ECHO_T}$_am_result" >&6
-rm -f confinc confmf
-
-# Check whether --enable-dependency-tracking or --disable-dependency-tracking was given.
-if test "${enable_dependency_tracking+set}" = set; then
- enableval="$enable_dependency_tracking"
-
-fi;
-if test "x$enable_dependency_tracking" != xno; then
- am_depcomp="$ac_aux_dir/depcomp"
- AMDEPBACKSLASH='\'
-fi
-
-
-if test "x$enable_dependency_tracking" != xno; then
- AMDEP_TRUE=
- AMDEP_FALSE='#'
-else
- AMDEP_TRUE='#'
- AMDEP_FALSE=
-fi
-
-
-
-
-depcc="$CC" am_compiler_list=
-
-echo "$as_me:$LINENO: checking dependency style of $depcc" >&5
-echo $ECHO_N "checking dependency style of $depcc... $ECHO_C" >&6
-if test "${am_cv_CC_dependencies_compiler_type+set}" = set; then
- echo $ECHO_N "(cached) $ECHO_C" >&6
-else
- if test -z "$AMDEP_TRUE" && test -f "$am_depcomp"; then
- # We make a subdir and do the tests there. Otherwise we can end up
- # making bogus files that we don't know about and never remove. For
- # instance it was reported that on HP-UX the gcc test will end up
- # making a dummy file named `D' -- because `-MD' means `put the output
- # in D'.
- mkdir conftest.dir
- # Copy depcomp to subdir because otherwise we won't find it if we're
- # using a relative directory.
- cp "$am_depcomp" conftest.dir
- cd conftest.dir
- # We will build objects and dependencies in a subdirectory because
- # it helps to detect inapplicable dependency modes. For instance
- # both Tru64's cc and ICC support -MD to output dependencies as a
- # side effect of compilation, but ICC will put the dependencies in
- # the current directory while Tru64 will put them in the object
- # directory.
- mkdir sub
-
- am_cv_CC_dependencies_compiler_type=none
- if test "$am_compiler_list" = ""; then
- am_compiler_list=`sed -n 's/^#*\([a-zA-Z0-9]*\))$/\1/p' < ./depcomp`
- fi
- for depmode in $am_compiler_list; do
- # Setup a source with many dependencies, because some compilers
- # like to wrap large dependency lists on column 80 (with \), and
- # we should not choose a depcomp mode which is confused by this.
- #
- # We need to recreate these files for each test, as the compiler may
- # overwrite some of them when testing with obscure command lines.
- # This happens at least with the AIX C compiler.
- : > sub/conftest.c
- for i in 1 2 3 4 5 6; do
- echo '#include "conftst'$i'.h"' >> sub/conftest.c
- # Using `: > sub/conftst$i.h' creates only sub/conftst1.h with
- # Solaris 8's {/usr,}/bin/sh.
- touch sub/conftst$i.h
- done
- echo "${am__include} ${am__quote}sub/conftest.Po${am__quote}" > confmf
-
- case $depmode in
- nosideeffect)
- # after this tag, mechanisms are not by side-effect, so they'll
- # only be used when explicitly requested
- if test "x$enable_dependency_tracking" = xyes; then
- continue
- else
- break
- fi
- ;;
- none) break ;;
- esac
- # We check with `-c' and `-o' for the sake of the "dashmstdout"
- # mode. It turns out that the SunPro C++ compiler does not properly
- # handle `-M -o', and we need to detect this.
- if depmode=$depmode \
- source=sub/conftest.c object=sub/conftest.${OBJEXT-o} \
- depfile=sub/conftest.Po tmpdepfile=sub/conftest.TPo \
- $SHELL ./depcomp $depcc -c -o sub/conftest.${OBJEXT-o} sub/conftest.c \
- >/dev/null 2>conftest.err &&
- grep sub/conftst6.h sub/conftest.Po > /dev/null 2>&1 &&
- grep sub/conftest.${OBJEXT-o} sub/conftest.Po > /dev/null 2>&1 &&
- ${MAKE-make} -s -f confmf > /dev/null 2>&1; then
- # icc doesn't choke on unknown options, it will just issue warnings
- # or remarks (even with -Werror). So we grep stderr for any message
- # that says an option was ignored or not supported.
- # When given -MP, icc 7.0 and 7.1 complain thusly:
- # icc: Command line warning: ignoring option '-M'; no argument required
- # The diagnosis changed in icc 8.0:
- # icc: Command line remark: option '-MP' not supported
- if (grep 'ignoring option' conftest.err ||
- grep 'not supported' conftest.err) >/dev/null 2>&1; then :; else
- am_cv_CC_dependencies_compiler_type=$depmode
- break
- fi
- fi
- done
-
- cd ..
- rm -rf conftest.dir
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ :
else
- am_cv_CC_dependencies_compiler_type=none
-fi
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+continue
fi
-echo "$as_me:$LINENO: result: $am_cv_CC_dependencies_compiler_type" >&5
-echo "${ECHO_T}$am_cv_CC_dependencies_compiler_type" >&6
-CCDEPMODE=depmode=$am_cv_CC_dependencies_compiler_type
-
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_declaration
+int
+main ()
+{
+exit (42);
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ break
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+done
+rm -f conftest*
+if test -n "$ac_declaration"; then
+ echo '#ifdef __cplusplus' >>confdefs.h
+ echo $ac_declaration >>confdefs.h
+ echo '#endif' >>confdefs.h
+fi
-if
- test "x$enable_dependency_tracking" != xno \
- && test "$am_cv_CC_dependencies_compiler_type" = gcc3; then
- am__fastdepCC_TRUE=
- am__fastdepCC_FALSE='#'
else
- am__fastdepCC_TRUE='#'
- am__fastdepCC_FALSE=
-fi
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ac_ext=c
+ac_cpp='$CPP $CPPFLAGS'
+ac_compile='$CC -c $CFLAGS $CPPFLAGS conftest.$ac_ext >&5'
+ac_link='$CC -o conftest$ac_exeext $CFLAGS $CPPFLAGS $LDFLAGS conftest.$ac_ext $LIBS >&5'
+ac_compiler_gnu=$ac_cv_c_compiler_gnu
# Check whether --enable-shared or --disable-shared was given.
if test "${enable_shared+set}" = set; then
@@ -3632,7 +4727,6 @@ LTCFLAGS=${LTCFLAGS-"$CFLAGS"}
# Allow CC to be a program name with arguments.
compiler=$CC
-
# Check whether --enable-libtool-lock or --disable-libtool-lock was given.
if test "${enable_libtool_lock+set}" = set; then
enableval="$enable_libtool_lock"
@@ -3664,7 +4758,7 @@ ia64-*-hpux*)
;;
*-*-irix6*)
# Find out which ABI we are using.
- echo '#line 3667 "configure"' > conftest.$ac_ext
+ echo '#line 4761 "configure"' > conftest.$ac_ext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
(eval $ac_compile) 2>&5
ac_status=$?
@@ -5263,7 +6357,7 @@ fi
# Provide some information about the compiler.
-echo "$as_me:5266:" \
+echo "$as_me:6360:" \
"checking for Fortran 77 compiler version" >&5
ac_compiler=`set X $ac_compile; echo $2`
{ (eval echo "$as_me:$LINENO: \"$ac_compiler --version &5\"") >&5
@@ -6326,11 +7420,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:6329: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:7423: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:6333: \$? = $ac_status" >&5
+ echo "$as_me:7427: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -6594,11 +7688,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:6597: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:7691: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:6601: \$? = $ac_status" >&5
+ echo "$as_me:7695: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -6698,11 +7792,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:6701: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:7795: $lt_compile\"" >&5)
(eval "$lt_compile" 2>out/conftest.err)
ac_status=$?
cat out/conftest.err >&5
- echo "$as_me:6705: \$? = $ac_status" >&5
+ echo "$as_me:7799: \$? = $ac_status" >&5
if (exit $ac_status) && test -s out/conftest2.$ac_objext
then
# The compiler can only warn and ignore the option if not recognized
@@ -8167,7 +9261,7 @@ linux*)
libsuff=
case "$host_cpu" in
x86_64*|s390x*|powerpc64*)
- echo '#line 8170 "configure"' > conftest.$ac_ext
+ echo '#line 9264 "configure"' > conftest.$ac_ext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
(eval $ac_compile) 2>&5
ac_status=$?
@@ -9064,7 +10158,7 @@ else
lt_dlunknown=0; lt_dlno_uscore=1; lt_dlneed_uscore=2
lt_status=$lt_dlunknown
cat > conftest.$ac_ext < conftest.$ac_ext <&5)
+ (eval echo "\"\$as_me:12604: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:11514: \$? = $ac_status" >&5
+ echo "$as_me:12608: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -11611,11 +12705,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:11614: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:12708: $lt_compile\"" >&5)
(eval "$lt_compile" 2>out/conftest.err)
ac_status=$?
cat out/conftest.err >&5
- echo "$as_me:11618: \$? = $ac_status" >&5
+ echo "$as_me:12712: \$? = $ac_status" >&5
if (exit $ac_status) && test -s out/conftest2.$ac_objext
then
# The compiler can only warn and ignore the option if not recognized
@@ -12147,7 +13241,7 @@ linux*)
libsuff=
case "$host_cpu" in
x86_64*|s390x*|powerpc64*)
- echo '#line 12150 "configure"' > conftest.$ac_ext
+ echo '#line 13244 "configure"' > conftest.$ac_ext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
(eval $ac_compile) 2>&5
ac_status=$?
@@ -13205,11 +14299,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:13208: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:14302: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:13212: \$? = $ac_status" >&5
+ echo "$as_me:14306: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -13309,11 +14403,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:13312: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:14406: $lt_compile\"" >&5)
(eval "$lt_compile" 2>out/conftest.err)
ac_status=$?
cat out/conftest.err >&5
- echo "$as_me:13316: \$? = $ac_status" >&5
+ echo "$as_me:14410: \$? = $ac_status" >&5
if (exit $ac_status) && test -s out/conftest2.$ac_objext
then
# The compiler can only warn and ignore the option if not recognized
@@ -14758,7 +15852,7 @@ linux*)
libsuff=
case "$host_cpu" in
x86_64*|s390x*|powerpc64*)
- echo '#line 14761 "configure"' > conftest.$ac_ext
+ echo '#line 15855 "configure"' > conftest.$ac_ext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
(eval $ac_compile) 2>&5
ac_status=$?
@@ -15536,11 +16630,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:15539: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:16633: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:15543: \$? = $ac_status" >&5
+ echo "$as_me:16637: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -15804,11 +16898,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:15807: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:16901: $lt_compile\"" >&5)
(eval "$lt_compile" 2>conftest.err)
ac_status=$?
cat conftest.err >&5
- echo "$as_me:15811: \$? = $ac_status" >&5
+ echo "$as_me:16905: \$? = $ac_status" >&5
if (exit $ac_status) && test -s "$ac_outfile"; then
# The compiler can only warn and ignore the option if not recognized
# So say no if there are warnings other than the usual output.
@@ -15908,11 +17002,11 @@ else
-e 's:.*FLAGS}\{0,1\} :&$lt_compiler_flag :; t' \
-e 's: [^ ]*conftest\.: $lt_compiler_flag&:; t' \
-e 's:$: $lt_compiler_flag:'`
- (eval echo "\"\$as_me:15911: $lt_compile\"" >&5)
+ (eval echo "\"\$as_me:17005: $lt_compile\"" >&5)
(eval "$lt_compile" 2>out/conftest.err)
ac_status=$?
cat out/conftest.err >&5
- echo "$as_me:15915: \$? = $ac_status" >&5
+ echo "$as_me:17009: \$? = $ac_status" >&5
if (exit $ac_status) && test -s out/conftest2.$ac_objext
then
# The compiler can only warn and ignore the option if not recognized
@@ -17377,7 +18471,7 @@ linux*)
libsuff=
case "$host_cpu" in
x86_64*|s390x*|powerpc64*)
- echo '#line 17380 "configure"' > conftest.$ac_ext
+ echo '#line 18474 "configure"' > conftest.$ac_ext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
(eval $ac_compile) 2>&5
ac_status=$?
@@ -19087,18 +20181,176 @@ fi
done
-JNI_CPPFLAGS=""
-if test $JAVA_HOME != ""
-then
- for dir in `find $JAVA_HOME/include -follow -type d`
- do
- JNI_CPPFLAGS="$JNI_CPPFLAGS -I$dir"
- done
-fi
-cppflags_bak=$CPPFLAGS
-CPPFLAGS="$CPPFLAGS $JNI_CPPFLAGS"
+JNI_CPPFLAGS=""
+if test $JAVA_HOME != ""
+then
+ for dir in `find $JAVA_HOME/include -follow -type d`
+ do
+ JNI_CPPFLAGS="$JNI_CPPFLAGS -I$dir"
+ done
+fi
+cppflags_bak=$CPPFLAGS
+CPPFLAGS="$CPPFLAGS $JNI_CPPFLAGS"
+
+for ac_header in jni.h
+do
+as_ac_Header=`echo "ac_cv_header_$ac_header" | $as_tr_sh`
+if eval "test \"\${$as_ac_Header+set}\" = set"; then
+ echo "$as_me:$LINENO: checking for $ac_header" >&5
+echo $ECHO_N "checking for $ac_header... $ECHO_C" >&6
+if eval "test \"\${$as_ac_Header+set}\" = set"; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+fi
+echo "$as_me:$LINENO: result: `eval echo '${'$as_ac_Header'}'`" >&5
+echo "${ECHO_T}`eval echo '${'$as_ac_Header'}'`" >&6
+else
+ # Is the header compilable?
+echo "$as_me:$LINENO: checking $ac_header usability" >&5
+echo $ECHO_N "checking $ac_header usability... $ECHO_C" >&6
+cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_includes_default
+#include <$ac_header>
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_header_compiler=yes
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+ac_header_compiler=no
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+echo "$as_me:$LINENO: result: $ac_header_compiler" >&5
+echo "${ECHO_T}$ac_header_compiler" >&6
+
+# Is the header present?
+echo "$as_me:$LINENO: checking $ac_header presence" >&5
+echo $ECHO_N "checking $ac_header presence... $ECHO_C" >&6
+cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+#include <$ac_header>
+_ACEOF
+if { (eval echo "$as_me:$LINENO: \"$ac_cpp conftest.$ac_ext\"") >&5
+ (eval $ac_cpp conftest.$ac_ext) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } >/dev/null; then
+ if test -s conftest.err; then
+ ac_cpp_err=$ac_c_preproc_warn_flag
+ ac_cpp_err=$ac_cpp_err$ac_c_werror_flag
+ else
+ ac_cpp_err=
+ fi
+else
+ ac_cpp_err=yes
+fi
+if test -z "$ac_cpp_err"; then
+ ac_header_preproc=yes
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+ ac_header_preproc=no
+fi
+rm -f conftest.err conftest.$ac_ext
+echo "$as_me:$LINENO: result: $ac_header_preproc" >&5
+echo "${ECHO_T}$ac_header_preproc" >&6
+
+# So? What about this header?
+case $ac_header_compiler:$ac_header_preproc:$ac_c_preproc_warn_flag in
+ yes:no: )
+ { echo "$as_me:$LINENO: WARNING: $ac_header: accepted by the compiler, rejected by the preprocessor!" >&5
+echo "$as_me: WARNING: $ac_header: accepted by the compiler, rejected by the preprocessor!" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: proceeding with the compiler's result" >&5
+echo "$as_me: WARNING: $ac_header: proceeding with the compiler's result" >&2;}
+ ac_header_preproc=yes
+ ;;
+ no:yes:* )
+ { echo "$as_me:$LINENO: WARNING: $ac_header: present but cannot be compiled" >&5
+echo "$as_me: WARNING: $ac_header: present but cannot be compiled" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: check for missing prerequisite headers?" >&5
+echo "$as_me: WARNING: $ac_header: check for missing prerequisite headers?" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: see the Autoconf documentation" >&5
+echo "$as_me: WARNING: $ac_header: see the Autoconf documentation" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: section \"Present But Cannot Be Compiled\"" >&5
+echo "$as_me: WARNING: $ac_header: section \"Present But Cannot Be Compiled\"" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: proceeding with the preprocessor's result" >&5
+echo "$as_me: WARNING: $ac_header: proceeding with the preprocessor's result" >&2;}
+ { echo "$as_me:$LINENO: WARNING: $ac_header: in the future, the compiler will take precedence" >&5
+echo "$as_me: WARNING: $ac_header: in the future, the compiler will take precedence" >&2;}
+ (
+ cat <<\_ASBOX
+## ------------------------------------------ ##
+## Report this to the AC_PACKAGE_NAME lists. ##
+## ------------------------------------------ ##
+_ASBOX
+ ) |
+ sed "s/^/$as_me: WARNING: /" >&2
+ ;;
+esac
+echo "$as_me:$LINENO: checking for $ac_header" >&5
+echo $ECHO_N "checking for $ac_header... $ECHO_C" >&6
+if eval "test \"\${$as_ac_Header+set}\" = set"; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ eval "$as_ac_Header=\$ac_header_preproc"
+fi
+echo "$as_me:$LINENO: result: `eval echo '${'$as_ac_Header'}'`" >&5
+echo "${ECHO_T}`eval echo '${'$as_ac_Header'}'`" >&6
+
+fi
+if test `eval echo '${'$as_ac_Header'}'` = yes; then
+ cat >>confdefs.h <<_ACEOF
+#define `echo "HAVE_$ac_header" | $as_tr_cpp` 1
+_ACEOF
+
+else
+ { { echo "$as_me:$LINENO: error: Native java headers not found. Is \$JAVA_HOME set correctly?" >&5
+echo "$as_me: error: Native java headers not found. Is \$JAVA_HOME set correctly?" >&2;}
+ { (exit 1); exit 1; }; }
+fi
+
+done
+
+CPPFLAGS=$cppflags_bak
+
+
+
-for ac_header in jni.h
+for ac_header in zlib.h zconf.h
do
as_ac_Header=`echo "ac_cv_header_$ac_header" | $as_tr_sh`
if eval "test \"\${$as_ac_Header+set}\" = set"; then
@@ -19243,20 +20495,54 @@ if test `eval echo '${'$as_ac_Header'}'` = yes; then
#define `echo "HAVE_$ac_header" | $as_tr_cpp` 1
_ACEOF
+echo "$as_me:$LINENO: checking Checking for the 'actual' dynamic-library for '-lz'" >&5
+echo $ECHO_N "checking Checking for the 'actual' dynamic-library for '-lz'... $ECHO_C" >&6
+if test "${ac_cv_libname_z+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
else
- { { echo "$as_me:$LINENO: error: Native java headers not found. Is \$JAVA_HOME set correctly?" >&5
-echo "$as_me: error: Native java headers not found. Is \$JAVA_HOME set correctly?" >&2;}
+
+ echo 'int main(int argc, char **argv){return 0;}' > conftest.c
+ if test -z "`${CC} ${LDFLAGS} -o conftest conftest.c -lz 2>&1`"; then
+ if test ! -z "`which objdump | grep -v 'no objdump'`"; then
+ ac_cv_libname_z="`objdump -p conftest | grep NEEDED | grep z | sed 's/\W*NEEDED\W*\(.*\)\W*$/\"\1\"/'`"
+ elif test ! -z "`which ldd | grep -v 'no ldd'`"; then
+ ac_cv_libname_z="`ldd conftest | grep z | sed 's/^[^A-Za-z0-9]*\([A-Za-z0-9\.]*\)[^A-Za-z0-9]*=>.*$/\"\1\"/'`"
+ elif test ! -z "`which otool | grep -v 'no otool'`"; then
+ ac_cv_libname_z=\"`otool -L conftest | grep z | sed -e 's/^ *//' -e 's/ .*//' -e 's/.*\/\(.*\)$/\1/'`\";
+ else
+ { { echo "$as_me:$LINENO: error: Can't find either 'objdump' or 'ldd' or 'otool' to compute the dynamic library for '-lz'" >&5
+echo "$as_me: error: Can't find either 'objdump' or 'ldd' or 'otool' to compute the dynamic library for '-lz'" >&2;}
+ { (exit 1); exit 1; }; }
+ fi
+ else
+ ac_cv_libname_z=libnotfound.so
+ fi
+ rm -f conftest*
+
+
+fi
+echo "$as_me:$LINENO: result: $ac_cv_libname_z" >&5
+echo "${ECHO_T}$ac_cv_libname_z" >&6
+
+cat >>confdefs.h <<_ACEOF
+#define HADOOP_ZLIB_LIBRARY ${ac_cv_libname_z}
+_ACEOF
+
+
+else
+ { { echo "$as_me:$LINENO: error: Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package." >&5
+echo "$as_me: error: Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package." >&2;}
{ (exit 1); exit 1; }; }
fi
done
-CPPFLAGS=$cppflags_bak
-for ac_header in zlib.h zconf.h
+
+for ac_header in fcntl.h stdlib.h string.h unistd.h
do
as_ac_Header=`echo "ac_cv_header_$ac_header" | $as_tr_sh`
if eval "test \"\${$as_ac_Header+set}\" = set"; then
@@ -19401,74 +20687,81 @@ if test `eval echo '${'$as_ac_Header'}'` = yes; then
#define `echo "HAVE_$ac_header" | $as_tr_cpp` 1
_ACEOF
-echo "$as_me:$LINENO: checking Checking for the 'actual' dynamic-library for '-lz'" >&5
-echo $ECHO_N "checking Checking for the 'actual' dynamic-library for '-lz'... $ECHO_C" >&6
-if test "${ac_cv_libname_z+set}" = set; then
- echo $ECHO_N "(cached) $ECHO_C" >&6
-else
-
- echo 'int main(int argc, char **argv){return 0;}' > conftest.c
- if test -z "`${CC} ${LDFLAGS} -o conftest conftest.c -lz 2>&1`"; then
- if test ! -z "`which objdump | grep -v 'no objdump'`"; then
- ac_cv_libname_z="`objdump -p conftest | grep NEEDED | grep z | sed 's/\W*NEEDED\W*\(.*\)\W*$/\"\1\"/'`"
- elif test ! -z "`which ldd | grep -v 'no ldd'`"; then
- ac_cv_libname_z="`ldd conftest | grep z | sed 's/^[^A-Za-z0-9]*\([A-Za-z0-9\.]*\)[^A-Za-z0-9]*=>.*$/\"\1\"/'`"
- else
- { { echo "$as_me:$LINENO: error: Can't find either 'objdump' or 'ldd' to compute the dynamic library for '-lz'" >&5
-echo "$as_me: error: Can't find either 'objdump' or 'ldd' to compute the dynamic library for '-lz'" >&2;}
- { (exit 1); exit 1; }; }
- fi
- else
- ac_cv_libname_z=libnotfound.so
- fi
- rm -f conftest*
-
-
-fi
-echo "$as_me:$LINENO: result: $ac_cv_libname_z" >&5
-echo "${ECHO_T}$ac_cv_libname_z" >&6
-
-cat >>confdefs.h <<_ACEOF
-#define HADOOP_ZLIB_LIBRARY ${ac_cv_libname_z}
-_ACEOF
-
-
else
- { { echo "$as_me:$LINENO: error: Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package." >&5
-echo "$as_me: error: Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package." >&2;}
+ { { echo "$as_me:$LINENO: error: Some system headers not found... please ensure their presence on your platform." >&5
+echo "$as_me: error: Some system headers not found... please ensure their presence on your platform." >&2;}
{ (exit 1); exit 1; }; }
fi
done
-
-
-
-
-for ac_header in fcntl.h stdlib.h string.h unistd.h
-do
-as_ac_Header=`echo "ac_cv_header_$ac_header" | $as_tr_sh`
-if eval "test \"\${$as_ac_Header+set}\" = set"; then
- echo "$as_me:$LINENO: checking for $ac_header" >&5
-echo $ECHO_N "checking for $ac_header... $ECHO_C" >&6
-if eval "test \"\${$as_ac_Header+set}\" = set"; then
+# Checks for typedefs, structures, and compiler characteristics.
+echo "$as_me:$LINENO: checking for an ANSI C-conforming const" >&5
+echo $ECHO_N "checking for an ANSI C-conforming const... $ECHO_C" >&6
+if test "${ac_cv_c_const+set}" = set; then
echo $ECHO_N "(cached) $ECHO_C" >&6
-fi
-echo "$as_me:$LINENO: result: `eval echo '${'$as_ac_Header'}'`" >&5
-echo "${ECHO_T}`eval echo '${'$as_ac_Header'}'`" >&6
else
- # Is the header compilable?
-echo "$as_me:$LINENO: checking $ac_header usability" >&5
-echo $ECHO_N "checking $ac_header usability... $ECHO_C" >&6
-cat >conftest.$ac_ext <<_ACEOF
+ cat >conftest.$ac_ext <<_ACEOF
/* confdefs.h. */
_ACEOF
cat confdefs.h >>conftest.$ac_ext
cat >>conftest.$ac_ext <<_ACEOF
/* end confdefs.h. */
-$ac_includes_default
-#include <$ac_header>
+
+int
+main ()
+{
+/* FIXME: Include the comments suggested by Paul. */
+#ifndef __cplusplus
+ /* Ultrix mips cc rejects this. */
+ typedef int charset[2];
+ const charset x;
+ /* SunOS 4.1.1 cc rejects this. */
+ char const *const *ccp;
+ char **p;
+ /* NEC SVR4.0.2 mips cc rejects this. */
+ struct point {int x, y;};
+ static struct point const zero = {0,0};
+ /* AIX XL C 1.02.0.0 rejects this.
+ It does not let you subtract one const X* pointer from another in
+ an arm of an if-expression whose if-part is not a constant
+ expression */
+ const char *g = "string";
+ ccp = &g + (g ? g-g : 0);
+ /* HPUX 7.0 cc rejects these. */
+ ++ccp;
+ p = (char**) ccp;
+ ccp = (char const *const *) p;
+ { /* SCO 3.2v4 cc rejects this. */
+ char *t;
+ char const *s = 0 ? (char *) 0 : (char const *) 0;
+
+ *t++ = 0;
+ }
+ { /* Someone thinks the Sun supposedly-ANSI compiler will reject this. */
+ int x[] = {25, 17};
+ const int *foo = &x[0];
+ ++foo;
+ }
+ { /* Sun SC1.0 ANSI compiler rejects this -- but not the above. */
+ typedef const int *iptr;
+ iptr p = 0;
+ ++p;
+ }
+ { /* AIX XL C 1.02.0.0 rejects this saying
+ "k.c", line 2.27: 1506-025 (S) Operand must be a modifiable lvalue. */
+ struct s { int j; const int *ap[3]; };
+ struct s *b; b->j = 5;
+ }
+ { /* ULTRIX-32 V3.1 (Rev 9) vcc rejects this */
+ const int foo = 10;
+ }
+#endif
+
+ ;
+ return 0;
+}
_ACEOF
rm -f conftest.$ac_objext
if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
@@ -19492,118 +20785,134 @@ if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
ac_status=$?
echo "$as_me:$LINENO: \$? = $ac_status" >&5
(exit $ac_status); }; }; then
- ac_header_compiler=yes
+ ac_cv_c_const=yes
else
echo "$as_me: failed program was:" >&5
sed 's/^/| /' conftest.$ac_ext >&5
-ac_header_compiler=no
+ac_cv_c_const=no
fi
rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
-echo "$as_me:$LINENO: result: $ac_header_compiler" >&5
-echo "${ECHO_T}$ac_header_compiler" >&6
+fi
+echo "$as_me:$LINENO: result: $ac_cv_c_const" >&5
+echo "${ECHO_T}$ac_cv_c_const" >&6
+if test $ac_cv_c_const = no; then
-# Is the header present?
-echo "$as_me:$LINENO: checking $ac_header presence" >&5
-echo $ECHO_N "checking $ac_header presence... $ECHO_C" >&6
-cat >conftest.$ac_ext <<_ACEOF
+cat >>confdefs.h <<\_ACEOF
+#define const
+_ACEOF
+
+fi
+
+
+# Checks for library functions.
+
+for ac_func in memset
+do
+as_ac_var=`echo "ac_cv_func_$ac_func" | $as_tr_sh`
+echo "$as_me:$LINENO: checking for $ac_func" >&5
+echo $ECHO_N "checking for $ac_func... $ECHO_C" >&6
+if eval "test \"\${$as_ac_var+set}\" = set"; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+ cat >conftest.$ac_ext <<_ACEOF
/* confdefs.h. */
_ACEOF
cat confdefs.h >>conftest.$ac_ext
cat >>conftest.$ac_ext <<_ACEOF
/* end confdefs.h. */
-#include <$ac_header>
+/* Define $ac_func to an innocuous variant, in case declares $ac_func.
+ For example, HP-UX 11i declares gettimeofday. */
+#define $ac_func innocuous_$ac_func
+
+/* System header to define __stub macros and hopefully few prototypes,
+ which can conflict with char $ac_func (); below.
+ Prefer to if __STDC__ is defined, since
+ exists even on freestanding compilers. */
+
+#ifdef __STDC__
+# include
+#else
+# include
+#endif
+
+#undef $ac_func
+
+/* Override any gcc2 internal prototype to avoid an error. */
+#ifdef __cplusplus
+extern "C"
+{
+#endif
+/* We use char because int might match the return type of a gcc2
+ builtin and then its argument prototype would still apply. */
+char $ac_func ();
+/* The GNU C library defines this for functions which it implements
+ to always fail with ENOSYS. Some functions are actually named
+ something starting with __ and the normal name is an alias. */
+#if defined (__stub_$ac_func) || defined (__stub___$ac_func)
+choke me
+#else
+char (*f) () = $ac_func;
+#endif
+#ifdef __cplusplus
+}
+#endif
+
+int
+main ()
+{
+return f != $ac_func;
+ ;
+ return 0;
+}
_ACEOF
-if { (eval echo "$as_me:$LINENO: \"$ac_cpp conftest.$ac_ext\"") >&5
- (eval $ac_cpp conftest.$ac_ext) 2>conftest.er1
+rm -f conftest.$ac_objext conftest$ac_exeext
+if { (eval echo "$as_me:$LINENO: \"$ac_link\"") >&5
+ (eval $ac_link) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest$ac_exeext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
ac_status=$?
- grep -v '^ *+' conftest.er1 >conftest.err
- rm -f conftest.er1
- cat conftest.err >&5
echo "$as_me:$LINENO: \$? = $ac_status" >&5
- (exit $ac_status); } >/dev/null; then
- if test -s conftest.err; then
- ac_cpp_err=$ac_c_preproc_warn_flag
- ac_cpp_err=$ac_cpp_err$ac_c_werror_flag
- else
- ac_cpp_err=
- fi
-else
- ac_cpp_err=yes
-fi
-if test -z "$ac_cpp_err"; then
- ac_header_preproc=yes
+ (exit $ac_status); }; }; then
+ eval "$as_ac_var=yes"
else
echo "$as_me: failed program was:" >&5
sed 's/^/| /' conftest.$ac_ext >&5
- ac_header_preproc=no
-fi
-rm -f conftest.err conftest.$ac_ext
-echo "$as_me:$LINENO: result: $ac_header_preproc" >&5
-echo "${ECHO_T}$ac_header_preproc" >&6
-
-# So? What about this header?
-case $ac_header_compiler:$ac_header_preproc:$ac_c_preproc_warn_flag in
- yes:no: )
- { echo "$as_me:$LINENO: WARNING: $ac_header: accepted by the compiler, rejected by the preprocessor!" >&5
-echo "$as_me: WARNING: $ac_header: accepted by the compiler, rejected by the preprocessor!" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: proceeding with the compiler's result" >&5
-echo "$as_me: WARNING: $ac_header: proceeding with the compiler's result" >&2;}
- ac_header_preproc=yes
- ;;
- no:yes:* )
- { echo "$as_me:$LINENO: WARNING: $ac_header: present but cannot be compiled" >&5
-echo "$as_me: WARNING: $ac_header: present but cannot be compiled" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: check for missing prerequisite headers?" >&5
-echo "$as_me: WARNING: $ac_header: check for missing prerequisite headers?" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: see the Autoconf documentation" >&5
-echo "$as_me: WARNING: $ac_header: see the Autoconf documentation" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: section \"Present But Cannot Be Compiled\"" >&5
-echo "$as_me: WARNING: $ac_header: section \"Present But Cannot Be Compiled\"" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: proceeding with the preprocessor's result" >&5
-echo "$as_me: WARNING: $ac_header: proceeding with the preprocessor's result" >&2;}
- { echo "$as_me:$LINENO: WARNING: $ac_header: in the future, the compiler will take precedence" >&5
-echo "$as_me: WARNING: $ac_header: in the future, the compiler will take precedence" >&2;}
- (
- cat <<\_ASBOX
-## ------------------------------------------ ##
-## Report this to the AC_PACKAGE_NAME lists. ##
-## ------------------------------------------ ##
-_ASBOX
- ) |
- sed "s/^/$as_me: WARNING: /" >&2
- ;;
-esac
-echo "$as_me:$LINENO: checking for $ac_header" >&5
-echo $ECHO_N "checking for $ac_header... $ECHO_C" >&6
-if eval "test \"\${$as_ac_Header+set}\" = set"; then
- echo $ECHO_N "(cached) $ECHO_C" >&6
-else
- eval "$as_ac_Header=\$ac_header_preproc"
+eval "$as_ac_var=no"
fi
-echo "$as_me:$LINENO: result: `eval echo '${'$as_ac_Header'}'`" >&5
-echo "${ECHO_T}`eval echo '${'$as_ac_Header'}'`" >&6
-
+rm -f conftest.err conftest.$ac_objext \
+ conftest$ac_exeext conftest.$ac_ext
fi
-if test `eval echo '${'$as_ac_Header'}'` = yes; then
+echo "$as_me:$LINENO: result: `eval echo '${'$as_ac_var'}'`" >&5
+echo "${ECHO_T}`eval echo '${'$as_ac_var'}'`" >&6
+if test `eval echo '${'$as_ac_var'}'` = yes; then
cat >>confdefs.h <<_ACEOF
-#define `echo "HAVE_$ac_header" | $as_tr_cpp` 1
+#define `echo "HAVE_$ac_func" | $as_tr_cpp` 1
_ACEOF
-else
- { { echo "$as_me:$LINENO: error: Some system headers not found... please ensure their presence on your platform." >&5
-echo "$as_me: error: Some system headers not found... please ensure their presence on your platform." >&2;}
- { (exit 1); exit 1; }; }
fi
-
done
-# Checks for typedefs, structures, and compiler characteristics.
-echo "$as_me:$LINENO: checking for an ANSI C-conforming const" >&5
-echo $ECHO_N "checking for an ANSI C-conforming const... $ECHO_C" >&6
-if test "${ac_cv_c_const+set}" = set; then
+# Check for nonstandard STRERROR_R
+echo "$as_me:$LINENO: checking whether strerror_r is declared" >&5
+echo $ECHO_N "checking whether strerror_r is declared... $ECHO_C" >&6
+if test "${ac_cv_have_decl_strerror_r+set}" = set; then
echo $ECHO_N "(cached) $ECHO_C" >&6
else
cat >conftest.$ac_ext <<_ACEOF
@@ -19612,55 +20921,12 @@ _ACEOF
cat confdefs.h >>conftest.$ac_ext
cat >>conftest.$ac_ext <<_ACEOF
/* end confdefs.h. */
-
+$ac_includes_default
int
main ()
{
-/* FIXME: Include the comments suggested by Paul. */
-#ifndef __cplusplus
- /* Ultrix mips cc rejects this. */
- typedef int charset[2];
- const charset x;
- /* SunOS 4.1.1 cc rejects this. */
- char const *const *ccp;
- char **p;
- /* NEC SVR4.0.2 mips cc rejects this. */
- struct point {int x, y;};
- static struct point const zero = {0,0};
- /* AIX XL C 1.02.0.0 rejects this.
- It does not let you subtract one const X* pointer from another in
- an arm of an if-expression whose if-part is not a constant
- expression */
- const char *g = "string";
- ccp = &g + (g ? g-g : 0);
- /* HPUX 7.0 cc rejects these. */
- ++ccp;
- p = (char**) ccp;
- ccp = (char const *const *) p;
- { /* SCO 3.2v4 cc rejects this. */
- char *t;
- char const *s = 0 ? (char *) 0 : (char const *) 0;
-
- *t++ = 0;
- }
- { /* Someone thinks the Sun supposedly-ANSI compiler will reject this. */
- int x[] = {25, 17};
- const int *foo = &x[0];
- ++foo;
- }
- { /* Sun SC1.0 ANSI compiler rejects this -- but not the above. */
- typedef const int *iptr;
- iptr p = 0;
- ++p;
- }
- { /* AIX XL C 1.02.0.0 rejects this saying
- "k.c", line 2.27: 1506-025 (S) Operand must be a modifiable lvalue. */
- struct s { int j; const int *ap[3]; };
- struct s *b; b->j = 5;
- }
- { /* ULTRIX-32 V3.1 (Rev 9) vcc rejects this */
- const int foo = 10;
- }
+#ifndef strerror_r
+ char *p = (char *) strerror_r;
#endif
;
@@ -19689,29 +20955,35 @@ if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
ac_status=$?
echo "$as_me:$LINENO: \$? = $ac_status" >&5
(exit $ac_status); }; }; then
- ac_cv_c_const=yes
+ ac_cv_have_decl_strerror_r=yes
else
echo "$as_me: failed program was:" >&5
sed 's/^/| /' conftest.$ac_ext >&5
-ac_cv_c_const=no
+ac_cv_have_decl_strerror_r=no
fi
rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
fi
-echo "$as_me:$LINENO: result: $ac_cv_c_const" >&5
-echo "${ECHO_T}$ac_cv_c_const" >&6
-if test $ac_cv_c_const = no; then
+echo "$as_me:$LINENO: result: $ac_cv_have_decl_strerror_r" >&5
+echo "${ECHO_T}$ac_cv_have_decl_strerror_r" >&6
+if test $ac_cv_have_decl_strerror_r = yes; then
-cat >>confdefs.h <<\_ACEOF
-#define const
+cat >>confdefs.h <<_ACEOF
+#define HAVE_DECL_STRERROR_R 1
+_ACEOF
+
+
+else
+ cat >>confdefs.h <<_ACEOF
+#define HAVE_DECL_STRERROR_R 0
_ACEOF
+
fi
-# Checks for library functions.
-for ac_func in memset
+for ac_func in strerror_r
do
as_ac_var=`echo "ac_cv_func_$ac_func" | $as_tr_sh`
echo "$as_me:$LINENO: checking for $ac_func" >&5
@@ -19812,6 +21084,123 @@ _ACEOF
fi
done
+echo "$as_me:$LINENO: checking whether strerror_r returns char *" >&5
+echo $ECHO_N "checking whether strerror_r returns char *... $ECHO_C" >&6
+if test "${ac_cv_func_strerror_r_char_p+set}" = set; then
+ echo $ECHO_N "(cached) $ECHO_C" >&6
+else
+
+ ac_cv_func_strerror_r_char_p=no
+ if test $ac_cv_have_decl_strerror_r = yes; then
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_includes_default
+int
+main ()
+{
+
+ char buf[100];
+ char x = *strerror_r (0, buf, sizeof buf);
+ char *p = strerror_r (0, buf, sizeof buf);
+
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest.$ac_objext
+if { (eval echo "$as_me:$LINENO: \"$ac_compile\"") >&5
+ (eval $ac_compile) 2>conftest.er1
+ ac_status=$?
+ grep -v '^ *+' conftest.er1 >conftest.err
+ rm -f conftest.er1
+ cat conftest.err >&5
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } &&
+ { ac_try='test -z "$ac_c_werror_flag"
+ || test ! -s conftest.err'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; } &&
+ { ac_try='test -s conftest.$ac_objext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_func_strerror_r_char_p=yes
+else
+ echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f conftest.err conftest.$ac_objext conftest.$ac_ext
+ else
+ # strerror_r is not declared. Choose between
+ # systems that have relatively inaccessible declarations for the
+ # function. BeOS and DEC UNIX 4.0 fall in this category, but the
+ # former has a strerror_r that returns char*, while the latter
+ # has a strerror_r that returns `int'.
+ # This test should segfault on the DEC system.
+ if test "$cross_compiling" = yes; then
+ :
+else
+ cat >conftest.$ac_ext <<_ACEOF
+/* confdefs.h. */
+_ACEOF
+cat confdefs.h >>conftest.$ac_ext
+cat >>conftest.$ac_ext <<_ACEOF
+/* end confdefs.h. */
+$ac_includes_default
+ extern char *strerror_r ();
+int
+main ()
+{
+char buf[100];
+ char x = *strerror_r (0, buf, sizeof buf);
+ exit (!isalpha (x));
+ ;
+ return 0;
+}
+_ACEOF
+rm -f conftest$ac_exeext
+if { (eval echo "$as_me:$LINENO: \"$ac_link\"") >&5
+ (eval $ac_link) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); } && { ac_try='./conftest$ac_exeext'
+ { (eval echo "$as_me:$LINENO: \"$ac_try\"") >&5
+ (eval $ac_try) 2>&5
+ ac_status=$?
+ echo "$as_me:$LINENO: \$? = $ac_status" >&5
+ (exit $ac_status); }; }; then
+ ac_cv_func_strerror_r_char_p=yes
+else
+ echo "$as_me: program exited with status $ac_status" >&5
+echo "$as_me: failed program was:" >&5
+sed 's/^/| /' conftest.$ac_ext >&5
+
+fi
+rm -f core *.core gmon.out bb.out conftest$ac_exeext conftest.$ac_objext conftest.$ac_ext
+fi
+ fi
+
+fi
+echo "$as_me:$LINENO: result: $ac_cv_func_strerror_r_char_p" >&5
+echo "${ECHO_T}$ac_cv_func_strerror_r_char_p" >&6
+if test $ac_cv_func_strerror_r_char_p = yes; then
+
+cat >>confdefs.h <<\_ACEOF
+#define STRERROR_R_CHAR_P 1
+_ACEOF
+
+fi
+
ac_config_files="$ac_config_files Makefile"
@@ -20459,6 +21848,13 @@ s,@ECHO_C@,$ECHO_C,;t t
s,@ECHO_N@,$ECHO_N,;t t
s,@ECHO_T@,$ECHO_T,;t t
s,@LIBS@,$LIBS,;t t
+s,@CC@,$CC,;t t
+s,@CFLAGS@,$CFLAGS,;t t
+s,@LDFLAGS@,$LDFLAGS,;t t
+s,@CPPFLAGS@,$CPPFLAGS,;t t
+s,@ac_ct_CC@,$ac_ct_CC,;t t
+s,@EXEEXT@,$EXEEXT,;t t
+s,@OBJEXT@,$OBJEXT,;t t
s,@INSTALL_PROGRAM@,$INSTALL_PROGRAM,;t t
s,@INSTALL_SCRIPT@,$INSTALL_SCRIPT,;t t
s,@INSTALL_DATA@,$INSTALL_DATA,;t t
@@ -20481,13 +21877,6 @@ s,@am__leading_dot@,$am__leading_dot,;t t
s,@AMTAR@,$AMTAR,;t t
s,@am__tar@,$am__tar,;t t
s,@am__untar@,$am__untar,;t t
-s,@CC@,$CC,;t t
-s,@CFLAGS@,$CFLAGS,;t t
-s,@LDFLAGS@,$LDFLAGS,;t t
-s,@CPPFLAGS@,$CPPFLAGS,;t t
-s,@ac_ct_CC@,$ac_ct_CC,;t t
-s,@EXEEXT@,$EXEEXT,;t t
-s,@OBJEXT@,$OBJEXT,;t t
s,@DEPDIR@,$DEPDIR,;t t
s,@am__include@,$am__include,;t t
s,@am__quote@,$am__quote,;t t
diff --git a/src/native/configure.ac b/src/native/configure.ac
index df118338761c1..94c86093b0c91 100644
--- a/src/native/configure.ac
+++ b/src/native/configure.ac
@@ -38,6 +38,7 @@ AC_INIT(src/org_apache_hadoop.h)
AC_CONFIG_SRCDIR([src/org_apache_hadoop.h])
AC_CONFIG_AUX_DIR(config)
AC_CONFIG_HEADER([config.h])
+AC_SYS_LARGEFILE
AM_INIT_AUTOMAKE(hadoop,1.0.0)
@@ -95,6 +96,9 @@ AC_C_CONST
# Checks for library functions.
AC_CHECK_FUNCS([memset])
+# Check for nonstandard STRERROR_R
+AC_FUNC_STRERROR_R
+
AC_CONFIG_FILES([Makefile])
AC_OUTPUT
diff --git a/src/native/lib/Makefile.am b/src/native/lib/Makefile.am
index 5d51cc5b76368..f33df0e9a857f 100644
--- a/src/native/lib/Makefile.am
+++ b/src/native/lib/Makefile.am
@@ -36,7 +36,7 @@ AM_LDFLAGS = @JNI_LDFLAGS@ -m$(JVM_DATA_MODEL)
lib_LTLIBRARIES = libhadoop.la
libhadoop_la_SOURCES =
-libhadoop_la_LDFLAGS = -version-info 1:0:0
+libhadoop_la_LDFLAGS = -version-info 1:0:0 $(AM_LDFLAGS)
libhadoop_la_LIBADD = $(HADOOP_OBJS) -ldl -ljvm
#
diff --git a/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c b/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
new file mode 100644
index 0000000000000..1c1a9896274c0
--- /dev/null
+++ b/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
@@ -0,0 +1,277 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+// get the autoconf settings
+#include "config.h"
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include "org_apache_hadoop.h"
+#include "org_apache_hadoop_io_nativeio_NativeIO.h"
+#include "file_descriptor.h"
+#include "errno_enum.h"
+
+// the NativeIO$Stat inner class and its constructor
+static jclass stat_clazz;
+static jmethodID stat_ctor;
+
+// the NativeIOException class and its constructor
+static jclass nioe_clazz;
+static jmethodID nioe_ctor;
+
+// Internal functions
+static void throw_ioe(JNIEnv* env, int errnum);
+static ssize_t get_pw_buflen();
+
+
+static void stat_init(JNIEnv *env) {
+ // Init Stat
+ jclass clazz = (*env)->FindClass(env, "org/apache/hadoop/io/nativeio/NativeIO$Stat");
+ PASS_EXCEPTIONS(env);
+ stat_clazz = (*env)->NewGlobalRef(env, clazz);
+ stat_ctor = (*env)->GetMethodID(env, stat_clazz, "",
+ "(Ljava/lang/String;Ljava/lang/String;I)V");
+}
+
+static void stat_deinit(JNIEnv *env) {
+ if (stat_clazz != NULL) {
+ (*env)->DeleteGlobalRef(env, stat_clazz);
+ stat_clazz = NULL;
+ }
+}
+
+static void nioe_init(JNIEnv *env) {
+ // Init NativeIOException
+ nioe_clazz = (*env)->FindClass(
+ env, "org/apache/hadoop/io/nativeio/NativeIOException");
+ PASS_EXCEPTIONS(env);
+
+ nioe_clazz = (*env)->NewGlobalRef(env, nioe_clazz);
+ nioe_ctor = (*env)->GetMethodID(env, nioe_clazz, "",
+ "(Ljava/lang/String;Lorg/apache/hadoop/io/nativeio/Errno;)V");
+}
+
+static void nioe_deinit(JNIEnv *env) {
+ if (nioe_clazz != NULL) {
+ (*env)->DeleteGlobalRef(env, nioe_clazz);
+ nioe_clazz = NULL;
+ }
+ nioe_ctor = NULL;
+}
+
+/*
+ * private static native void initNative();
+ *
+ * We rely on this function rather than lazy initialization because
+ * the lazy approach may have a race if multiple callers try to
+ * init at the same time.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_initNative(
+ JNIEnv *env, jclass clazz) {
+
+ stat_init(env);
+ PASS_EXCEPTIONS_GOTO(env, error);
+ nioe_init(env);
+ PASS_EXCEPTIONS_GOTO(env, error);
+ fd_init(env);
+ PASS_EXCEPTIONS_GOTO(env, error);
+ errno_enum_init(env);
+ PASS_EXCEPTIONS_GOTO(env, error);
+ return;
+error:
+ // these are all idempodent and safe to call even if the
+ // class wasn't initted yet
+ stat_deinit(env);
+ nioe_deinit(env);
+ fd_deinit(env);
+ errno_enum_deinit(env);
+}
+
+/*
+ * public static native Stat fstat(FileDescriptor fd);
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_fstat(
+ JNIEnv *env, jclass clazz, jobject fd_object)
+{
+ jobject ret = NULL;
+ char *pw_buf = NULL;
+
+ int fd = fd_get(env, fd_object);
+ PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+ struct stat s;
+ int rc = fstat(fd, &s);
+ if (rc != 0) {
+ throw_ioe(env, errno);
+ goto cleanup;
+ }
+
+ size_t pw_buflen = get_pw_buflen();
+ if ((pw_buf = malloc(pw_buflen)) == NULL) {
+ THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
+ goto cleanup;
+ }
+
+ // Grab username
+ struct passwd pwd, *pwdp;
+ while ((rc = getpwuid_r(s.st_uid, &pwd, pw_buf, pw_buflen, &pwdp)) != 0) {
+ if (rc != ERANGE) {
+ throw_ioe(env, rc);
+ goto cleanup;
+ }
+ free(pw_buf);
+ pw_buflen *= 2;
+ if ((pw_buf = malloc(pw_buflen)) == NULL) {
+ THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
+ goto cleanup;
+ }
+ }
+ assert(pwdp == &pwd);
+
+ jstring jstr_username = (*env)->NewStringUTF(env, pwd.pw_name);
+ if (jstr_username == NULL) goto cleanup;
+
+ // Grab group
+ struct group grp, *grpp;
+ while ((rc = getgrgid_r(s.st_gid, &grp, pw_buf, pw_buflen, &grpp)) != 0) {
+ if (rc != ERANGE) {
+ throw_ioe(env, rc);
+ goto cleanup;
+ }
+ free(pw_buf);
+ pw_buflen *= 2;
+ if ((pw_buf = malloc(pw_buflen)) == NULL) {
+ THROW(env, "java/lang/OutOfMemoryError", "Couldn't allocate memory for pw buffer");
+ goto cleanup;
+ }
+ }
+ assert(grpp == &grp);
+
+ jstring jstr_groupname = (*env)->NewStringUTF(env, grp.gr_name);
+ PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+ // Construct result
+ ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
+ jstr_username, jstr_groupname, s.st_mode);
+
+cleanup:
+ if (pw_buf != NULL) free(pw_buf);
+ return ret;
+}
+
+
+/*
+ * public static native FileDescriptor open(String path, int flags, int mode);
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_open(
+ JNIEnv *env, jclass clazz, jstring j_path,
+ jint flags, jint mode)
+{
+ jobject ret = NULL;
+
+ const char *path = (*env)->GetStringUTFChars(env, j_path, NULL);
+ if (path == NULL) goto cleanup; // JVM throws Exception for us
+
+ int fd;
+ if (flags & O_CREAT) {
+ fd = open(path, flags, mode);
+ } else {
+ fd = open(path, flags);
+ }
+
+ if (fd == -1) {
+ throw_ioe(env, errno);
+ goto cleanup;
+ }
+
+ ret = fd_create(env, fd);
+
+cleanup:
+ if (path != NULL) {
+ (*env)->ReleaseStringUTFChars(env, j_path, path);
+ }
+ return ret;
+}
+
+/*
+ * Throw a java.IO.IOException, generating the message from errno.
+ */
+static void throw_ioe(JNIEnv* env, int errnum)
+{
+ const char* message;
+ char buffer[80];
+ jstring jstr_message;
+
+ buffer[0] = 0;
+#ifdef STRERROR_R_CHAR_P
+ // GNU strerror_r
+ message = strerror_r(errnum, buffer, sizeof(buffer));
+ assert (message != NULL);
+#else
+ int ret = strerror_r(errnum, buffer, sizeof(buffer));
+ if (ret == 0) {
+ message = buffer;
+ } else {
+ message = "Unknown error";
+ }
+#endif
+ jobject errno_obj = errno_to_enum(env, errnum);
+
+ if ((jstr_message = (*env)->NewStringUTF(env, message)) == NULL)
+ goto err;
+
+ jthrowable obj = (jthrowable)(*env)->NewObject(env, nioe_clazz, nioe_ctor,
+ jstr_message, errno_obj);
+ if (obj == NULL) goto err;
+
+ (*env)->Throw(env, obj);
+ return;
+
+err:
+ if (jstr_message != NULL)
+ (*env)->ReleaseStringUTFChars(env, jstr_message, message);
+}
+
+
+/*
+ * Determine how big a buffer we need for reentrant getpwuid_r and getgrnam_r
+ */
+ssize_t get_pw_buflen() {
+ size_t ret = 0;
+ #ifdef _SC_GETPW_R_SIZE_MAX
+ ret = sysconf(_SC_GETPW_R_SIZE_MAX);
+ #endif
+ return (ret > 512) ? ret : 512;
+}
+/**
+ * vim: sw=2: ts=2: et:
+ */
+
diff --git a/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.c b/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.c
new file mode 100644
index 0000000000000..76d1ff172521f
--- /dev/null
+++ b/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.c
@@ -0,0 +1,119 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+ #include
+ #include
+ #include
+
+#include "org_apache_hadoop.h"
+
+typedef struct errno_mapping {
+ int errno_val;
+ char *errno_str;
+} errno_mapping_t;
+
+// Macro to define structs like {FOO, "FOO"} for each errno value
+#define MAPPING(x) {x, #x}
+static errno_mapping_t ERRNO_MAPPINGS[] = {
+ MAPPING(EPERM),
+ MAPPING(ENOENT),
+ MAPPING(ESRCH),
+ MAPPING(EINTR),
+ MAPPING(EIO),
+ MAPPING(ENXIO),
+ MAPPING(E2BIG),
+ MAPPING(ENOEXEC),
+ MAPPING(EBADF),
+ MAPPING(ECHILD),
+ MAPPING(EAGAIN),
+ MAPPING(ENOMEM),
+ MAPPING(EACCES),
+ MAPPING(EFAULT),
+ MAPPING(ENOTBLK),
+ MAPPING(EBUSY),
+ MAPPING(EEXIST),
+ MAPPING(EXDEV),
+ MAPPING(ENODEV),
+ MAPPING(ENOTDIR),
+ MAPPING(EISDIR),
+ MAPPING(EINVAL),
+ MAPPING(ENFILE),
+ MAPPING(EMFILE),
+ MAPPING(ENOTTY),
+ MAPPING(ETXTBSY),
+ MAPPING(EFBIG),
+ MAPPING(ENOSPC),
+ MAPPING(ESPIPE),
+ MAPPING(EROFS),
+ MAPPING(EMLINK),
+ MAPPING(EPIPE),
+ MAPPING(EDOM),
+ MAPPING(ERANGE),
+ {-1, NULL}
+};
+
+static jclass enum_class;
+static jmethodID enum_valueOf;
+static jclass errno_class;
+
+void errno_enum_init(JNIEnv *env) {
+ if (enum_class != NULL) return;
+
+ enum_class = (*env)->FindClass(env, "java/lang/Enum");
+ PASS_EXCEPTIONS(env);
+ enum_class = (*env)->NewGlobalRef(env, enum_class);
+ enum_valueOf = (*env)->GetStaticMethodID(env, enum_class,
+ "valueOf", "(Ljava/lang/Class;Ljava/lang/String;)Ljava/lang/Enum;");
+ PASS_EXCEPTIONS(env);
+
+ errno_class = (*env)->FindClass(env, "org/apache/hadoop/io/nativeio/Errno");
+ PASS_EXCEPTIONS(env);
+ errno_class = (*env)->NewGlobalRef(env, errno_class);
+}
+
+void errno_enum_deinit(JNIEnv *env) {
+ if (enum_class != NULL) {
+ (*env)->DeleteGlobalRef(env, enum_class);
+ enum_class = NULL;
+ }
+ if (errno_class != NULL) {
+ (*env)->DeleteGlobalRef(env, errno_class);
+ errno_class = NULL;
+ }
+ enum_valueOf = NULL;
+}
+
+
+static char *errno_to_string(int errnum) {
+ int i;
+ for (i = 0; ERRNO_MAPPINGS[i].errno_str != NULL; i++) {
+ if (ERRNO_MAPPINGS[i].errno_val == errnum)
+ return ERRNO_MAPPINGS[i].errno_str;
+ }
+ return "UNKNOWN";
+}
+
+jobject errno_to_enum(JNIEnv *env, int errnum) {
+ char *str = errno_to_string(errnum);
+ assert(str != NULL);
+
+ jstring jstr = (*env)->NewStringUTF(env, str);
+ PASS_EXCEPTIONS_RET(env, NULL);
+
+ return (*env)->CallStaticObjectMethod(
+ env, enum_class, enum_valueOf, errno_class, jstr);
+}
diff --git a/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.h b/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.h
new file mode 100644
index 0000000000000..1eee11a5f8283
--- /dev/null
+++ b/src/native/src/org/apache/hadoop/io/nativeio/errno_enum.h
@@ -0,0 +1,27 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+#ifndef ERRNO_ENUM_H
+#define ERRNO_ENUM_H
+
+#include
+
+void errno_enum_init(JNIEnv *env);
+void errno_enum_deinit(JNIEnv *env);
+jobject errno_to_enum(JNIEnv *env, int errnum);
+
+#endif
diff --git a/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.c b/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.c
new file mode 100644
index 0000000000000..0681db8f8324e
--- /dev/null
+++ b/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.c
@@ -0,0 +1,69 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+#include
+#include "file_descriptor.h"
+#include "org_apache_hadoop.h"
+
+// class of java.io.FileDescriptor
+static jclass fd_class;
+// the internal field for the integer fd
+static jfieldID fd_descriptor;
+// the no-argument constructor
+static jmethodID fd_constructor;
+
+
+void fd_init(JNIEnv* env)
+{
+ if (fd_class != NULL) return; // already initted
+
+ fd_class = (*env)->FindClass(env, "java/io/FileDescriptor");
+ PASS_EXCEPTIONS(env);
+ fd_class = (*env)->NewGlobalRef(env, fd_class);
+
+ fd_descriptor = (*env)->GetFieldID(env, fd_class, "fd", "I");
+ PASS_EXCEPTIONS(env);
+ fd_constructor = (*env)->GetMethodID(env, fd_class, "", "()V");
+}
+
+void fd_deinit(JNIEnv *env) {
+ if (fd_class != NULL) {
+ (*env)->DeleteGlobalRef(env, fd_class);
+ fd_class = NULL;
+ }
+ fd_descriptor = NULL;
+ fd_constructor = NULL;
+}
+
+/*
+ * Given an instance 'obj' of java.io.FileDescriptor, return the
+ * underlying fd, or throw if unavailable
+ */
+int fd_get(JNIEnv* env, jobject obj) {
+ return (*env)->GetIntField(env, obj, fd_descriptor);
+}
+
+/*
+ * Create a FileDescriptor object corresponding to the given int fd
+ */
+jobject fd_create(JNIEnv *env, int fd) {
+ jobject obj = (*env)->NewObject(env, fd_class, fd_constructor);
+ PASS_EXCEPTIONS_RET(env, NULL);
+
+ (*env)->SetIntField(env, obj, fd_descriptor, fd);
+ return obj;
+}
diff --git a/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.h b/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.h
new file mode 100644
index 0000000000000..3f689493bc595
--- /dev/null
+++ b/src/native/src/org/apache/hadoop/io/nativeio/file_descriptor.h
@@ -0,0 +1,28 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+#ifndef FILE_DESCRIPTOR_H
+#define FILE_DESCRIPTOR_H
+
+#include
+
+void fd_init(JNIEnv *env);
+void fd_deinit(JNIEnv *env);
+
+int fd_get(JNIEnv* env, jobject obj);
+jobject fd_create(JNIEnv *env, int fd);
+
+#endif
diff --git a/src/native/src/org_apache_hadoop.h b/src/native/src/org_apache_hadoop.h
index 325dcd6601fdf..c1513eeb8a991 100644
--- a/src/native/src/org_apache_hadoop.h
+++ b/src/native/src/org_apache_hadoop.h
@@ -50,6 +50,22 @@
} \
}
+/* Helper macro to return if an exception is pending */
+#define PASS_EXCEPTIONS(env) \
+ { \
+ if ((*env)->ExceptionCheck(env)) return; \
+ }
+
+#define PASS_EXCEPTIONS_GOTO(env, target) \
+ { \
+ if ((*env)->ExceptionCheck(env)) goto target; \
+ }
+
+#define PASS_EXCEPTIONS_RET(env, ret) \
+ { \
+ if ((*env)->ExceptionCheck(env)) return (ret); \
+ }
+
/**
* A helper function to dlsym a 'symbol' from a given library-handle.
*
diff --git a/src/protobuf/SerializationMetadata.proto b/src/protobuf/SerializationMetadata.proto
new file mode 100644
index 0000000000000..e4249632355eb
--- /dev/null
+++ b/src/protobuf/SerializationMetadata.proto
@@ -0,0 +1,15 @@
+package org.apache.hadoop.io.serial.lib;
+
+message TypedSerializationMetadata {
+ optional string typename = 1;
+}
+
+message AvroMetadata {
+ optional string schema = 1;
+ optional Kind kind = 2;
+ enum Kind {
+ SPECIFIC = 1;
+ GENERIC = 2;
+ REFLECTION = 3;
+ }
+}
\ No newline at end of file
diff --git a/src/test/core/org/apache/hadoop/io/AvroKey.java b/src/test/core/org/apache/hadoop/io/AvroKey.java
new file mode 100644
index 0000000000000..35e14885eace6
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/AvroKey.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.io;
+
+@SuppressWarnings("all")
+public class AvroKey extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+ public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"AvroKey\",\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]}");
+ public int value;
+ public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+ public java.lang.Object get(int field$) {
+ switch (field$) {
+ case 0: return value;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+ @SuppressWarnings(value="unchecked")
+ public void put(int field$, java.lang.Object value$) {
+ switch (field$) {
+ case 0: value = (java.lang.Integer)value$; break;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/AvroValue.java b/src/test/core/org/apache/hadoop/io/AvroValue.java
new file mode 100644
index 0000000000000..9da257bfc13e2
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/AvroValue.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.io;
+
+@SuppressWarnings("all")
+public class AvroValue extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+ public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"AvroValue\",\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{\"name\":\"value\",\"type\":\"string\"}]}");
+ public org.apache.avro.util.Utf8 value;
+ public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+ public java.lang.Object get(int field$) {
+ switch (field$) {
+ case 0: return value;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+ @SuppressWarnings(value="unchecked")
+ public void put(int field$, java.lang.Object value$) {
+ switch (field$) {
+ case 0: value = (org.apache.avro.util.Utf8)value$; break;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/ProtoTest.java b/src/test/core/org/apache/hadoop/io/ProtoTest.java
new file mode 100644
index 0000000000000..86bb526276fb9
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ProtoTest.java
@@ -0,0 +1,641 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: src/test/core/org/apache/hadoop/io/ProtoTest.proto
+
+package org.apache.hadoop.io;
+
+public final class ProtoTest {
+ private ProtoTest() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ public static final class ProtoKey extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ProtoKey.newBuilder() to construct.
+ private ProtoKey() {
+ initFields();
+ }
+ private ProtoKey(boolean noInit) {}
+
+ private static final ProtoKey defaultInstance;
+ public static ProtoKey getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ProtoKey getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoKey_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable;
+ }
+
+ // required int32 value = 1;
+ public static final int VALUE_FIELD_NUMBER = 1;
+ private boolean hasValue;
+ private int value_ = 0;
+ public boolean hasValue() { return hasValue; }
+ public int getValue() { return value_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasValue) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasValue()) {
+ output.writeInt32(1, getValue());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasValue()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(1, getValue());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.ProtoTest.ProtoKey prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private org.apache.hadoop.io.ProtoTest.ProtoKey result;
+
+ // Construct using org.apache.hadoop.io.ProtoTest.ProtoKey.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.ProtoTest.ProtoKey();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.ProtoTest.ProtoKey internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.ProtoTest.ProtoKey();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoKey.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoKey getDefaultInstanceForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoKey.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.ProtoTest.ProtoKey build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.ProtoTest.ProtoKey buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoKey buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.ProtoTest.ProtoKey returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.ProtoTest.ProtoKey) {
+ return mergeFrom((org.apache.hadoop.io.ProtoTest.ProtoKey)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.ProtoTest.ProtoKey other) {
+ if (other == org.apache.hadoop.io.ProtoTest.ProtoKey.getDefaultInstance()) return this;
+ if (other.hasValue()) {
+ setValue(other.getValue());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ setValue(input.readInt32());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required int32 value = 1;
+ public boolean hasValue() {
+ return result.hasValue();
+ }
+ public int getValue() {
+ return result.getValue();
+ }
+ public Builder setValue(int value) {
+ result.hasValue = true;
+ result.value_ = value;
+ return this;
+ }
+ public Builder clearValue() {
+ result.hasValue = false;
+ result.value_ = 0;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.ProtoKey)
+ }
+
+ static {
+ defaultInstance = new ProtoKey(true);
+ org.apache.hadoop.io.ProtoTest.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.ProtoKey)
+ }
+
+ public static final class ProtoValue extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ProtoValue.newBuilder() to construct.
+ private ProtoValue() {
+ initFields();
+ }
+ private ProtoValue(boolean noInit) {}
+
+ private static final ProtoValue defaultInstance;
+ public static ProtoValue getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ProtoValue getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoValue_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable;
+ }
+
+ // required string value = 2;
+ public static final int VALUE_FIELD_NUMBER = 2;
+ private boolean hasValue;
+ private java.lang.String value_ = "";
+ public boolean hasValue() { return hasValue; }
+ public java.lang.String getValue() { return value_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasValue) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasValue()) {
+ output.writeString(2, getValue());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasValue()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getValue());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.ProtoTest.ProtoValue prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private org.apache.hadoop.io.ProtoTest.ProtoValue result;
+
+ // Construct using org.apache.hadoop.io.ProtoTest.ProtoValue.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.ProtoTest.ProtoValue();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.ProtoTest.ProtoValue internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.ProtoTest.ProtoValue();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoValue.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoValue getDefaultInstanceForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoValue.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.ProtoTest.ProtoValue build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.ProtoTest.ProtoValue buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoValue buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.ProtoTest.ProtoValue returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.ProtoTest.ProtoValue) {
+ return mergeFrom((org.apache.hadoop.io.ProtoTest.ProtoValue)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.ProtoTest.ProtoValue other) {
+ if (other == org.apache.hadoop.io.ProtoTest.ProtoValue.getDefaultInstance()) return this;
+ if (other.hasValue()) {
+ setValue(other.getValue());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 18: {
+ setValue(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string value = 2;
+ public boolean hasValue() {
+ return result.hasValue();
+ }
+ public java.lang.String getValue() {
+ return result.getValue();
+ }
+ public Builder setValue(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasValue = true;
+ result.value_ = value;
+ return this;
+ }
+ public Builder clearValue() {
+ result.hasValue = false;
+ result.value_ = getDefaultInstance().getValue();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.ProtoValue)
+ }
+
+ static {
+ defaultInstance = new ProtoValue(true);
+ org.apache.hadoop.io.ProtoTest.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.ProtoValue)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n2src/test/core/org/apache/hadoop/io/Pro" +
+ "toTest.proto\022\024org.apache.hadoop.io\"\031\n\010Pr" +
+ "otoKey\022\r\n\005value\030\001 \002(\005\"\033\n\nProtoValue\022\r\n\005v" +
+ "alue\030\002 \002(\tB\002H\001"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor,
+ new java.lang.String[] { "Value", },
+ org.apache.hadoop.io.ProtoTest.ProtoKey.class,
+ org.apache.hadoop.io.ProtoTest.ProtoKey.Builder.class);
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor,
+ new java.lang.String[] { "Value", },
+ org.apache.hadoop.io.ProtoTest.ProtoValue.class,
+ org.apache.hadoop.io.ProtoTest.ProtoValue.Builder.class);
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ }, assigner);
+ }
+
+ public static void internalForceInit() {}
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/src/test/core/org/apache/hadoop/io/ProtoTest.proto b/src/test/core/org/apache/hadoop/io/ProtoTest.proto
new file mode 100644
index 0000000000000..0524bab2083e9
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ProtoTest.proto
@@ -0,0 +1,11 @@
+package org.apache.hadoop.io;
+
+option optimize_for = SPEED;
+
+message ProtoKey {
+ required int32 value = 1;
+}
+
+message ProtoValue {
+ required string value = 2;
+}
diff --git a/src/test/core/org/apache/hadoop/io/RandomDatum.java b/src/test/core/org/apache/hadoop/io/RandomDatum.java
index 5a4bbc0bcd1f9..558a38629a980 100644
--- a/src/test/core/org/apache/hadoop/io/RandomDatum.java
+++ b/src/test/core/org/apache/hadoop/io/RandomDatum.java
@@ -39,7 +39,7 @@ public int getLength() {
public void write(DataOutput out) throws IOException {
out.writeInt(length);
- out.write(data);
+ out.write(data, 0, length);
}
public void readFields(DataInput in) throws IOException {
diff --git a/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java b/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
index c96cc732938ef..0465cc101244a 100644
--- a/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
+++ b/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
@@ -26,6 +26,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.serial.lib.JavaSerialization;
+import org.apache.hadoop.io.serial.lib.WritableSerialization;
public class TestDefaultStringifier extends TestCase {
@@ -36,7 +39,8 @@ public class TestDefaultStringifier extends TestCase {
public void testWithWritable() throws Exception {
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.WritableSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ WritableSerialization.class.getName());
LOG.info("Testing DefaultStringifier with Text");
@@ -51,7 +55,8 @@ public void testWithWritable() throws Exception {
builder.append(alphabet[random.nextInt(alphabet.length)]);
}
Text text = new Text(builder.toString());
- DefaultStringifier stringifier = new DefaultStringifier(conf, Text.class);
+ DefaultStringifier stringifier =
+ new DefaultStringifier(conf, Text.class);
String str = stringifier.toString(text);
Text claimedText = stringifier.fromString(str);
@@ -62,13 +67,15 @@ public void testWithWritable() throws Exception {
}
public void testWithJavaSerialization() throws Exception {
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.JavaSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
LOG.info("Testing DefaultStringifier with Serializable Integer");
//Integer implements Serializable
Integer testInt = Integer.valueOf(42);
- DefaultStringifier stringifier = new DefaultStringifier(conf, Integer.class);
+ DefaultStringifier stringifier =
+ new DefaultStringifier(conf, Integer.class);
String str = stringifier.toString(testInt);
Integer claimedInt = stringifier.fromString(str);
@@ -80,7 +87,8 @@ public void testWithJavaSerialization() throws Exception {
public void testStoreLoad() throws IOException {
LOG.info("Testing DefaultStringifier#store() and #load()");
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.WritableSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ WritableSerialization.class.getName());
Text text = new Text("uninteresting test string");
String keyName = "test.defaultstringifier.key1";
@@ -94,7 +102,8 @@ public void testStoreLoad() throws IOException {
public void testStoreLoadArray() throws IOException {
LOG.info("Testing DefaultStringifier#storeArray() and #loadArray()");
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.JavaSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
String keyName = "test.defaultstringifier.key2";
diff --git a/src/test/core/org/apache/hadoop/io/TestMapFile.java b/src/test/core/org/apache/hadoop/io/TestMapFile.java
index f006d4f40131d..e4cdc25393519 100644
--- a/src/test/core/org/apache/hadoop/io/TestMapFile.java
+++ b/src/test/core/org/apache/hadoop/io/TestMapFile.java
@@ -36,10 +36,13 @@ public void testGetClosest() throws Exception {
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
// Make an index entry for every third insertion.
MapFile.Writer.setIndexInterval(conf, 3);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), Text.class, Text.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(Text.class),
+ MapFile.Writer.valueClass(Text.class));
// Assert that the index interval is 1
assertEquals(3, writer.getIndexInterval());
// Add entries up to 100 in intervals of ten.
@@ -51,8 +54,7 @@ public void testGetClosest() throws Exception {
}
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
Text key = new Text("55");
Text value = new Text();
Text closest = (Text)reader.getClosest(key, value);
@@ -94,14 +96,16 @@ public void testMidKey() throws Exception {
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), IntWritable.class, IntWritable.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(IntWritable.class),
+ MapFile.Writer.valueClass(IntWritable.class));
writer.append(new IntWritable(1), new IntWritable(1));
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
assertEquals(new IntWritable(1), reader.midKey());
}
@@ -112,13 +116,15 @@ public void testMidKeyEmpty() throws Exception {
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), IntWritable.class, IntWritable.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(IntWritable.class),
+ MapFile.Writer.valueClass(IntWritable.class));
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
assertEquals(null, reader.midKey());
}
}
diff --git a/src/test/core/org/apache/hadoop/io/TestSecureIOUtils.java b/src/test/core/org/apache/hadoop/io/TestSecureIOUtils.java
new file mode 100644
index 0000000000000..df87507adbd29
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/TestSecureIOUtils.java
@@ -0,0 +1,83 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.io;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+import org.junit.BeforeClass;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+
+public class TestSecureIOUtils {
+ private static String realOwner, realGroup;
+ private static final File testFilePath =
+ new File(System.getProperty("test.build.data"), "TestSecureIOContext");
+
+ @BeforeClass
+ public static void makeTestFile() throws Exception {
+ FileOutputStream fos = new FileOutputStream(testFilePath);
+ fos.write("hello".getBytes("UTF-8"));
+ fos.close();
+
+ Configuration conf = new Configuration();
+ FileSystem rawFS = FileSystem.getLocal(conf).getRaw();
+ FileStatus stat = rawFS.getFileStatus(
+ new Path(testFilePath.toString()));
+ realOwner = stat.getOwner();
+ realGroup = stat.getGroup();
+ }
+
+ @Test
+ public void testReadUnrestricted() throws IOException {
+ SecureIOUtils.openForRead(testFilePath, null, null).close();
+ }
+
+ @Test
+ public void testReadCorrectlyRestrictedWithSecurity() throws IOException {
+ SecureIOUtils
+ .openForRead(testFilePath, realOwner, realGroup).close();
+ }
+
+ @Test(expected=IOException.class)
+ public void testReadIncorrectlyRestrictedWithSecurity() throws IOException {
+ SecureIOUtils
+ .openForRead(testFilePath, "invalidUser", null).close();
+ fail("Didn't throw expection for wrong ownership!");
+ }
+
+ @Test
+ public void testCreateForWrite() throws IOException {
+ try {
+ SecureIOUtils.createForWrite(testFilePath, 0777);
+ fail("Was able to create file at " + testFilePath);
+ } catch (SecureIOUtils.AlreadyExistsException aee) {
+ // expected
+ }
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java b/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
index c9fc1eae4f5ae..53aec3b04d946 100644
--- a/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
+++ b/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
@@ -20,50 +20,314 @@
import junit.framework.TestCase;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.util.Utf8;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ProtoTest.ProtoKey;
+import org.apache.hadoop.io.ProtoTest.ProtoValue;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.SequenceFile.Reader;
import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.lib.CompatibilitySerialization;
+import org.apache.hadoop.io.serial.lib.JavaSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroSerialization.Kind;
public class TestSequenceFileSerialization extends TestCase {
private Configuration conf;
private FileSystem fs;
-
+ private Path file;
+
@Override
protected void setUp() throws Exception {
conf = new Configuration();
- conf.set("io.serializations",
- "org.apache.hadoop.io.serializer.JavaSerialization");
fs = FileSystem.getLocal(conf);
+ file = new Path(System.getProperty("test.build.data",".") + "/test.seq");
}
@Override
protected void tearDown() throws Exception {
fs.close();
}
-
- public void testJavaSerialization() throws Exception {
- Path file = new Path(System.getProperty("test.build.data",".") +
- "/test.seq");
+
+ public void writeAvroSpecificSerialization(CompressionType kind
+ ) throws Exception {
+ AvroKey key = new AvroKey();
+ AvroValue value = new AvroValue();
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(AvroKey.class),
+ SequenceFile.Writer.valueClass(AvroValue.class));
+ key.value = 1;
+ value.value = new Utf8("one");
+ writer.append(key, value);
+ key.value = 2;
+ value.value = new Utf8("two");
+ writer.append(key, value);
+ String writerKeySerialStr = writer.getKeySerialization().toString();
+ String writerValueSerialStr = writer.getValueSerialization().toString();
+ writer.close();
+
+ assertEquals("{schema: '{\"type\":\"record\",\"name\":\"AvroKey\"," +
+ "\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{" +
+ "\"name\":\"value\",\"type\":\"int\"}]}',\n" +
+ " kind: SPECIFIC}\n",
+ writerKeySerialStr);
+ assertEquals("{schema: '{\"type\":\"record\",\"name\":\"AvroValue\"," +
+ "\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{" +
+ "\"name\":\"value\",\"type\":\"string\"}]}',\n" +
+ " kind: SPECIFIC}\n",
+ writerValueSerialStr);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization> keySerialClone = factory.getSerialization("avro");
+ keySerialClone.fromString(writerKeySerialStr);
+ Serialization> valueSerialClone = factory.getSerialization("avro");
+ valueSerialClone.fromString(writerValueSerialStr);
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ Serialization> keySerial = reader.getKeySerialization();
+ Serialization> valueSerial = reader.getValueSerialization();
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("avro", keySerial.getName());
+ assertEquals(writerKeySerialStr, keySerial.toString());
+ assertEquals(keySerialClone, keySerial);
+ assertEquals("avro", valueSerial.getName());
+ assertEquals(writerValueSerialStr, valueSerial.toString());
+ assertEquals(valueSerialClone, valueSerial);
+
+ assertEquals(1, ((AvroKey) reader.nextKey(key)).value);
+ assertEquals(new Utf8("one"),
+ ((AvroValue) reader.getCurrentValue(value)).value);
+ assertEquals(2, ((AvroKey) reader.nextKey(key)).value);
+ assertEquals(new Utf8("two"),
+ ((AvroValue) reader.getCurrentValue(value)).value);
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void readAvroGenericSerialization() throws Exception {
+ Serialization> serial = new AvroSerialization(Kind.GENERIC);
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file),
+ SequenceFile.Reader.keySerialization(serial),
+ SequenceFile.Reader.valueSerialization(serial.clone()));
+
+ assertEquals(1, ((GenericRecord) reader.nextKey(null)).get("value"));
+ assertEquals(new Utf8("one"),
+ ((GenericRecord) reader.getCurrentValue(null)).get("value"));
+ assertEquals(2, ((GenericRecord) reader.nextKey(null)).get("value"));
+ assertEquals(new Utf8("two"),
+ ((GenericRecord) reader.getCurrentValue(null)).get("value"));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeProtobufSerialization(CompressionType kind
+ ) throws Exception {
fs.delete(file, true);
- Writer writer = SequenceFile.createWriter(fs, conf, file, Long.class,
- String.class);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(ProtoKey.class),
+ SequenceFile.Writer.valueClass(ProtoValue.class));
+ writer.append(ProtoKey.newBuilder().setValue(1).build(),
+ ProtoValue.newBuilder().setValue("one").build());
+ writer.append(ProtoKey.newBuilder().setValue(2).build(),
+ ProtoValue.newBuilder().setValue("two").build());
+ String keySerialStr = writer.getKeySerialization().toString();
+ assertEquals("{class: org.apache.hadoop.io.ProtoTest$ProtoKey}\n",
+ keySerialStr);
+ String valueSerialStr = writer.getValueSerialization().toString();
+ assertEquals("{class: org.apache.hadoop.io.ProtoTest$ProtoValue}\n",
+ valueSerialStr);
+ writer.close();
+
+ // build serializers from the string form
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization> keySerial = factory.getSerialization("protobuf");
+ keySerial.fromString(keySerialStr);
+ Serialization> valueSerial = factory.getSerialization("protobuf");
+ valueSerial.fromString(valueSerialStr);
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ Serialization> readerKeySerial = reader.getKeySerialization();
+ Serialization> readerValueSerial = reader.getValueSerialization();
+ assertEquals("protobuf", readerKeySerial.getName());
+ assertEquals(keySerialStr, readerKeySerial.toString());
+ assertEquals(keySerial, readerKeySerial);
+ assertEquals("protobuf", readerValueSerial.getName());
+ assertEquals(valueSerialStr, readerValueSerial.toString());
+ assertEquals(valueSerial, readerValueSerial);
+
+ assertEquals(ProtoKey.newBuilder().setValue(1).build(),
+ reader.nextKey(null));
+ assertEquals(ProtoValue.newBuilder().setValue("one").build(),
+ reader.getCurrentValue(null));
+ assertEquals(ProtoKey.newBuilder().setValue(2).build(),
+ reader.nextKey(null));
+ assertEquals(ProtoValue.newBuilder().setValue("two").build(),
+ reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeThriftSerialization(CompressionType kind) throws Exception {
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(ThriftKey.class),
+ SequenceFile.Writer.valueClass(ThriftValue.class));
+ writer.append(new ThriftKey(1), new ThriftValue("one"));
+ writer.append(new ThriftKey(2), new ThriftValue("two"));
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("thrift", reader.getKeySerialization().getName());
+ assertEquals("thrift", reader.getValueSerialization().getName());
+ assertEquals(new ThriftKey(1), reader.nextKey(null));
+ assertEquals(new ThriftValue("one"), reader.getCurrentValue(null));
+ assertEquals(new ThriftKey(2), reader.nextKey(null));
+ assertEquals(new ThriftValue("two"), reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeWritableSerialization(CompressionType kind
+ ) throws Exception {
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(IntWritable.class),
+ SequenceFile.Writer.valueClass(Text.class));
+ writer.append(new IntWritable(1), new Text("one"));
+ writer.append(new IntWritable(2), new Text("two"));
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("writable", reader.getKeySerialization().getName());
+ assertEquals("writable", reader.getValueSerialization().getName());
+ assertEquals(new IntWritable(1), reader.nextKey(null));
+ assertEquals(new Text("one"), reader.getCurrentValue(null));
+ assertEquals(new IntWritable(2), reader.nextKey(null));
+ assertEquals(new Text("two"), reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeJavaSerialization(CompressionType kind) throws Exception {
+ fs.delete(file, true);
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(Long.class),
+ SequenceFile.Writer.valueClass(String.class));
writer.append(1L, "one");
writer.append(2L, "two");
-
writer.close();
- Reader reader = new Reader(fs, file, conf);
- assertEquals(1L, reader.next((Object) null));
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("java", reader.getKeySerialization().getName());
+ assertEquals("java", reader.getValueSerialization().getName());
+ assertEquals(1L, reader.nextKey(null));
assertEquals("one", reader.getCurrentValue((Object) null));
- assertEquals(2L, reader.next((Object) null));
+ assertEquals(2L, reader.nextKey(null));
assertEquals("two", reader.getCurrentValue((Object) null));
- assertNull(reader.next((Object) null));
+ assertNull(reader.nextKey(null));
reader.close();
}
+
+ /**
+ * Test the compatibility layer to load the old java serialization.
+ */
+ public void writeOldJavaSerialization(CompressionType kind
+ ) throws Exception {
+ fs.delete(file, true);
+ // set the old attribute to include the java serialization
+ conf.set("io.serializations",
+ "org.apache.hadoop.io.serializer.JavaSerialization");
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization> serial = factory.getSerializationByType(Long.class);
+ assertEquals(CompatibilitySerialization.class, serial.getClass());
+
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(Long.class),
+ SequenceFile.Writer.valueClass(String.class));
+ writer.append(1L, "one");
+ writer.append(2L, "two");
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals("compatibility", reader.getKeySerialization().getName());
+ assertEquals("compatibility", reader.getValueSerialization().getName());
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals(1L, reader.nextKey(null));
+ assertEquals("one", reader.getCurrentValue((Object) null));
+ assertEquals(2L, reader.nextKey(null));
+ assertEquals("two", reader.getCurrentValue((Object) null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void testAvro() throws Exception {
+ writeAvroSpecificSerialization(CompressionType.NONE);
+ readAvroGenericSerialization();
+ writeAvroSpecificSerialization(CompressionType.RECORD);
+ writeAvroSpecificSerialization(CompressionType.BLOCK);
+ }
+
+ public void testProtobuf() throws Exception {
+ writeProtobufSerialization(CompressionType.NONE);
+ writeProtobufSerialization(CompressionType.RECORD);
+ writeProtobufSerialization(CompressionType.BLOCK);
+ }
+
+ public void testThrift() throws Exception {
+ writeThriftSerialization(CompressionType.NONE);
+ writeThriftSerialization(CompressionType.RECORD);
+ writeThriftSerialization(CompressionType.BLOCK);
+ }
+
+ public void testWritable() throws Exception {
+ writeWritableSerialization(CompressionType.NONE);
+ writeWritableSerialization(CompressionType.RECORD);
+ writeWritableSerialization(CompressionType.BLOCK);
+ }
+
+ public void testJava() throws Exception {
+ writeJavaSerialization(CompressionType.NONE);
+ writeJavaSerialization(CompressionType.RECORD);
+ writeJavaSerialization(CompressionType.BLOCK);
+ }
+
+ public void testOldJava() throws Exception {
+ writeOldJavaSerialization(CompressionType.NONE);
+ writeOldJavaSerialization(CompressionType.RECORD);
+ writeOldJavaSerialization(CompressionType.BLOCK);
+ }
}
diff --git a/src/test/core/org/apache/hadoop/io/ThriftKey.java b/src/test/core/org/apache/hadoop/io/ThriftKey.java
new file mode 100644
index 0000000000000..b4a3fe28edf27
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ThriftKey.java
@@ -0,0 +1,307 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.io;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ThriftKey implements TBase, java.io.Serializable, Cloneable {
+ private static final TStruct STRUCT_DESC = new TStruct("ThriftKey");
+
+ private static final TField VALUE_FIELD_DESC = new TField("value", TType.I32, (short)1);
+
+ public int value;
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map byName = new HashMap();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // VALUE
+ return VALUE;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __VALUE_ISSET_ID = 0;
+ private BitSet __isset_bit_vector = new BitSet(1);
+
+ public static final Map<_Fields, FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
+ new FieldValueMetaData(TType.I32)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ FieldMetaData.addStructMetaDataMap(ThriftKey.class, metaDataMap);
+ }
+
+ public ThriftKey() {
+ }
+
+ public ThriftKey(
+ int value)
+ {
+ this();
+ this.value = value;
+ setValueIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on other.
+ */
+ public ThriftKey(ThriftKey other) {
+ __isset_bit_vector.clear();
+ __isset_bit_vector.or(other.__isset_bit_vector);
+ this.value = other.value;
+ }
+
+ public ThriftKey deepCopy() {
+ return new ThriftKey(this);
+ }
+
+ @Override
+ public void clear() {
+ setValueIsSet(false);
+ this.value = 0;
+ }
+
+ public int getValue() {
+ return this.value;
+ }
+
+ public ThriftKey setValue(int value) {
+ this.value = value;
+ setValueIsSet(true);
+ return this;
+ }
+
+ public void unsetValue() {
+ __isset_bit_vector.clear(__VALUE_ISSET_ID);
+ }
+
+ /** Returns true if field value is set (has been asigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return __isset_bit_vector.get(__VALUE_ISSET_ID);
+ }
+
+ public void setValueIsSet(boolean value) {
+ __isset_bit_vector.set(__VALUE_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((Integer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return new Integer(getValue());
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof ThriftKey)
+ return this.equals((ThriftKey)that);
+ return false;
+ }
+
+ public boolean equals(ThriftKey that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true;
+ boolean that_present_value = true;
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (this.value != that.value)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(ThriftKey other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ ThriftKey typedOther = (ThriftKey)other;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = TBaseHelper.compareTo(this.value, typedOther.value);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id) {
+ case 1: // VALUE
+ if (field.type == TType.I32) {
+ this.value = iprot.readI32();
+ setValueIsSet(true);
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ validate();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeI32(this.value);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("ThriftKey(");
+ boolean first = true;
+
+ sb.append("value:");
+ sb.append(this.value);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws TException {
+ // check for required fields
+ }
+
+}
+
diff --git a/src/test/core/org/apache/hadoop/io/ThriftValue.java b/src/test/core/org/apache/hadoop/io/ThriftValue.java
new file mode 100644
index 0000000000000..efb2b4ad47fc9
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ThriftValue.java
@@ -0,0 +1,309 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.io;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ThriftValue implements TBase, java.io.Serializable, Cloneable {
+ private static final TStruct STRUCT_DESC = new TStruct("ThriftValue");
+
+ private static final TField VALUE_FIELD_DESC = new TField("value", TType.STRING, (short)1);
+
+ public String value;
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map byName = new HashMap();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // VALUE
+ return VALUE;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+
+ public static final Map<_Fields, FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
+ new FieldValueMetaData(TType.STRING)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ FieldMetaData.addStructMetaDataMap(ThriftValue.class, metaDataMap);
+ }
+
+ public ThriftValue() {
+ }
+
+ public ThriftValue(
+ String value)
+ {
+ this();
+ this.value = value;
+ }
+
+ /**
+ * Performs a deep copy on other.
+ */
+ public ThriftValue(ThriftValue other) {
+ if (other.isSetValue()) {
+ this.value = other.value;
+ }
+ }
+
+ public ThriftValue deepCopy() {
+ return new ThriftValue(this);
+ }
+
+ @Override
+ public void clear() {
+ this.value = null;
+ }
+
+ public String getValue() {
+ return this.value;
+ }
+
+ public ThriftValue setValue(String value) {
+ this.value = value;
+ return this;
+ }
+
+ public void unsetValue() {
+ this.value = null;
+ }
+
+ /** Returns true if field value is set (has been asigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return this.value != null;
+ }
+
+ public void setValueIsSet(boolean value) {
+ if (!value) {
+ this.value = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((String)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return getValue();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof ThriftValue)
+ return this.equals((ThriftValue)that);
+ return false;
+ }
+
+ public boolean equals(ThriftValue that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true && this.isSetValue();
+ boolean that_present_value = true && that.isSetValue();
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (!this.value.equals(that.value))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(ThriftValue other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ ThriftValue typedOther = (ThriftValue)other;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = TBaseHelper.compareTo(this.value, typedOther.value);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id) {
+ case 1: // VALUE
+ if (field.type == TType.STRING) {
+ this.value = iprot.readString();
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ validate();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (this.value != null) {
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeString(this.value);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("ThriftValue(");
+ boolean first = true;
+
+ sb.append("value:");
+ if (this.value == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.value);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws TException {
+ // check for required fields
+ }
+
+}
+
diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
index f47ec7034c503..8c219174bdc35 100644
--- a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
+++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
@@ -43,6 +43,7 @@ public void setUp() throws IOException {
}
}
+@SuppressWarnings("serial")
class MyComparator implements RawComparator, Serializable {
@Override
@@ -54,6 +55,6 @@ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
public int compare(byte[] o1, byte[] o2) {
return WritableComparator.compareBytes(o1, 0, o1.length, o2, 0, o2.length);
}
-
+
}
diff --git a/src/test/core/org/apache/hadoop/io/nativeio/TestNativeIO.java b/src/test/core/org/apache/hadoop/io/nativeio/TestNativeIO.java
new file mode 100644
index 0000000000000..972c9fc25c85a
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/nativeio/TestNativeIO.java
@@ -0,0 +1,137 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.io.nativeio;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assume.*;
+import static org.junit.Assert.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+public class TestNativeIO {
+ static final Log LOG = LogFactory.getLog(TestNativeIO.class);
+
+ static final File TEST_DIR = new File(
+ System.getProperty("test.build.data"), "testnativeio");
+
+ @Before
+ public void checkLoaded() {
+ assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
+ }
+
+ @Before
+ public void setupTestDir() throws IOException {
+ FileUtil.fullyDelete(TEST_DIR);
+ TEST_DIR.mkdirs();
+ }
+
+ @Test
+ public void testFstat() throws Exception {
+ FileOutputStream fos = new FileOutputStream(
+ new File(TEST_DIR, "testfstat"));
+ NativeIO.Stat stat = NativeIO.fstat(fos.getFD());
+ fos.close();
+ LOG.info("Stat: " + String.valueOf(stat));
+
+ assertEquals(System.getProperty("user.name"), stat.getOwner());
+ assertNotNull(stat.getGroup());
+ assertTrue(!"".equals(stat.getGroup()));
+ assertEquals("Stat mode field should indicate a regular file",
+ NativeIO.Stat.S_IFREG, stat.getMode() & NativeIO.Stat.S_IFMT);
+ }
+
+ @Test
+ public void testFstatClosedFd() throws Exception {
+ FileOutputStream fos = new FileOutputStream(
+ new File(TEST_DIR, "testfstat2"));
+ fos.close();
+ try {
+ NativeIO.Stat stat = NativeIO.fstat(fos.getFD());
+ } catch (NativeIOException nioe) {
+ LOG.info("Got expected exception", nioe);
+ assertEquals(Errno.EBADF, nioe.getErrno());
+ }
+ }
+
+ @Test
+ public void testOpenMissingWithoutCreate() throws Exception {
+ LOG.info("Open a missing file without O_CREAT and it should fail");
+ try {
+ FileDescriptor fd = NativeIO.open(
+ new File(TEST_DIR, "doesntexist").getAbsolutePath(),
+ NativeIO.O_WRONLY, 0700);
+ fail("Able to open a new file without O_CREAT");
+ } catch (NativeIOException nioe) {
+ LOG.info("Got expected exception", nioe);
+ assertEquals(Errno.ENOENT, nioe.getErrno());
+ }
+ }
+
+ @Test
+ public void testOpenWithCreate() throws Exception {
+ LOG.info("Test creating a file with O_CREAT");
+ FileDescriptor fd = NativeIO.open(
+ new File(TEST_DIR, "testWorkingOpen").getAbsolutePath(),
+ NativeIO.O_WRONLY | NativeIO.O_CREAT, 0700);
+ assertNotNull(true);
+ assertTrue(fd.valid());
+ FileOutputStream fos = new FileOutputStream(fd);
+ fos.write("foo".getBytes());
+ fos.close();
+
+ assertFalse(fd.valid());
+
+ LOG.info("Test exclusive create");
+ try {
+ fd = NativeIO.open(
+ new File(TEST_DIR, "testWorkingOpen").getAbsolutePath(),
+ NativeIO.O_WRONLY | NativeIO.O_CREAT | NativeIO.O_EXCL, 0700);
+ fail("Was able to create existing file with O_EXCL");
+ } catch (NativeIOException nioe) {
+ LOG.info("Got expected exception for failed exclusive create", nioe);
+ assertEquals(Errno.EEXIST, nioe.getErrno());
+ }
+ }
+
+ /**
+ * Test that opens and closes a file 10000 times - this would crash with
+ * "Too many open files" if we leaked fds using this access pattern.
+ */
+ @Test
+ public void testFDDoesntLeak() throws IOException {
+ for (int i = 0; i < 10000; i++) {
+ FileDescriptor fd = NativeIO.open(
+ new File(TEST_DIR, "testNoFdLeak").getAbsolutePath(),
+ NativeIO.O_WRONLY | NativeIO.O_CREAT, 0700);
+ assertNotNull(true);
+ assertTrue(fd.valid());
+ FileOutputStream fos = new FileOutputStream(fd);
+ fos.write("foo".getBytes());
+ fos.close();
+ }
+ }
+
+}
diff --git a/src/test/core/org/apache/hadoop/io/test.genavro b/src/test/core/org/apache/hadoop/io/test.genavro
new file mode 100644
index 0000000000000..bc694d9094ab8
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/test.genavro
@@ -0,0 +1,10 @@
+@namespace("org.apache.hadoop.io")
+protocol AvroTest {
+ record AvroKey {
+ int value;
+ }
+
+ record AvroValue {
+ string value;
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/test.thrift b/src/test/core/org/apache/hadoop/io/test.thrift
new file mode 100644
index 0000000000000..5858521d5f0d6
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/test.thrift
@@ -0,0 +1,7 @@
+namespace java org.apache.hadoop.io
+struct ThriftKey {
+ 1: i32 value
+}
+struct ThriftValue {
+ 1: string value
+}
diff --git a/src/test/findbugsExcludeFile.xml b/src/test/findbugsExcludeFile.xml
index 615f52179c9bf..a43c17fce3e71 100644
--- a/src/test/findbugsExcludeFile.xml
+++ b/src/test/findbugsExcludeFile.xml
@@ -192,6 +192,16 @@
+
+
+
+
+
+
+
+
+
+