You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by ma...@apache.org on 2010/02/06 01:56:52 UTC

svn commit: r907144 - in /hadoop/avro/trunk: ./ lang/c/ lang/c/src/ lang/c/tests/

Author: massie
Date: Sat Feb  6 00:56:51 2010
New Revision: 907144

URL: http://svn.apache.org/viewvc?rev=907144&view=rev
Log:
AVRO-403. Add file object container support to C implementation

Added:
    hadoop/avro/trunk/lang/c/src/datafile.c
    hadoop/avro/trunk/lang/c/tests/generate_interop_data.c
    hadoop/avro/trunk/lang/c/tests/test_interop_data.c
Modified:
    hadoop/avro/trunk/CHANGES.txt
    hadoop/avro/trunk/build.sh
    hadoop/avro/trunk/lang/c/build.sh
    hadoop/avro/trunk/lang/c/src/Makefile.am
    hadoop/avro/trunk/lang/c/src/avro.h
    hadoop/avro/trunk/lang/c/src/avro_private.h
    hadoop/avro/trunk/lang/c/src/io.c
    hadoop/avro/trunk/lang/c/tests/Makefile.am
    hadoop/avro/trunk/lang/c/version.sh

Modified: hadoop/avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/CHANGES.txt?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/CHANGES.txt (original)
+++ hadoop/avro/trunk/CHANGES.txt Sat Feb  6 00:56:51 2010
@@ -309,6 +309,8 @@
 
     AVRO-398. avro_read_file doesn't detect eof (Eli Collins via massie)
 
+    AVRO-403. Add file object container support to C implementation (massie)
+
   OPTIMIZATIONS
 
     AVRO-172. More efficient schema processing (massie)

Modified: hadoop/avro/trunk/build.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/build.sh?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/build.sh (original)
+++ hadoop/avro/trunk/build.sh Sat Feb  6 00:56:51 2010
@@ -49,12 +49,13 @@
 	# create interop test data
 	(cd lang/java; ant interop-data-generate)
 	#(cd lang/py; ant interop-data-generate)
-	#(cd lang/c; make interop-data-generate)
+	(cd lang/c; ./build.sh interop-data-generate)
 	#(cd lang/c++; make interop-data-generate)
 
 	# run interop data tests
 	(cd lang/java; ant interop-data-test)
 	#(cd lang/py; ant interop-data-test)
+	(cd lang/c; ./build.sh interop-data-test)
 	#(cd lang/c; make interop-data-test)
 	#(cd lang/c++; make interop-data-test)
 

Modified: hadoop/avro/trunk/lang/c/build.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/build.sh?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/build.sh (original)
+++ hadoop/avro/trunk/lang/c/build.sh Sat Feb  6 00:56:51 2010
@@ -31,6 +31,18 @@
 
 case "$1" in
 
+    interop-data-generate)
+	prepare_build
+	make -C $build_dir
+	$build_dir/tests/generate_interop_data "../../share/test/schemas/interop.avsc"  "../../build/interop/data"
+	;;
+
+    interop-data-test)
+	prepare_build
+	make -C $build_dir
+	$build_dir/tests/test_interop_data "../../build/interop/data"
+	;;
+
     test)
 	prepare_build
 	make -C $build_dir check
@@ -58,7 +70,7 @@
 	;;
 
     *)
-        echo "Usage: $0 {test|dist|clean}"
+        echo "Usage: $0 {interop-data-generate|interop-data-test|test|dist|clean}"
         exit 1
 esac
 

Modified: hadoop/avro/trunk/lang/c/src/Makefile.am
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/src/Makefile.am?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/src/Makefile.am (original)
+++ hadoop/avro/trunk/lang/c/src/Makefile.am Sat Feb  6 00:56:51 2010
@@ -9,7 +9,7 @@
 libavro_la_SOURCES = st.c st.h schema.c schema.h schema_equal.c \
 datum.c datum_equal.c datum_validate.c datum_read.c datum_skip.c datum_write.c datum.h \
 io.c dump.c dump.h encoding_binary.c \
-container_of.h encoding.h
+container_of.h encoding.h datafile.c
 libavro_la_LIBADD = $(top_builddir)/jansson/src/.libs/libjansson.a
 libavro_la_LDFLAGS = \
         -version-info $(LIBAVRO_VERSION) \

Modified: hadoop/avro/trunk/lang/c/src/avro.h
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/src/avro.h?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/src/avro.h (original)
+++ hadoop/avro/trunk/lang/c/src/avro.h Sat Feb  6 00:56:51 2010
@@ -258,5 +258,23 @@
 int avro_write_data(avro_writer_t writer,
 		    avro_schema_t writer_schema, avro_datum_t datum);
 
+/* File object container */
+typedef struct avro_file_reader_t *avro_file_reader_t;
+typedef struct avro_file_writer_t *avro_file_writer_t;
+
+int avro_file_writer_create(const char *path, avro_schema_t schema,
+			    avro_file_writer_t * writer);
+int avro_file_writer_open(const char *path, avro_file_writer_t * writer);
+int avro_file_reader(const char *path, avro_file_reader_t * reader);
+
+int avro_file_writer_append(avro_file_writer_t writer, avro_datum_t datum);
+int avro_file_writer_sync(avro_file_writer_t writer);
+int avro_file_writer_flush(avro_file_writer_t writer);
+int avro_file_writer_close(avro_file_writer_t writer);
+
+int avro_file_reader_read(avro_file_reader_t reader,
+			  avro_schema_t readers_schema, avro_datum_t * datum);
+int avro_file_reader_close(avro_file_reader_t reader);
+
 CLOSE_EXTERN
 #endif

Modified: hadoop/avro/trunk/lang/c/src/avro_private.h
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/src/avro_private.h?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/src/avro_private.h (original)
+++ hadoop/avro/trunk/lang/c/src/avro_private.h Sat Feb  6 00:56:51 2010
@@ -24,4 +24,8 @@
 #define container_of(ptr_, type_, member_)  \
     ((type_ *)((char *)ptr_ - (size_t)&((type_ *)0)->member_))
 
+void avro_writer_reset(avro_writer_t writer);
+int64_t avro_writer_tell(avro_writer_t writer);
+void avro_writer_flush(avro_writer_t writer);
+
 #endif

Added: hadoop/avro/trunk/lang/c/src/datafile.c
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/src/datafile.c?rev=907144&view=auto
==============================================================================
--- hadoop/avro/trunk/lang/c/src/datafile.c (added)
+++ hadoop/avro/trunk/lang/c/src/datafile.c Sat Feb  6 00:56:51 2010
@@ -0,0 +1,365 @@
+/*
+ * 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 <avro.h>
+#include "avro_private.h"
+#include "encoding.h"
+#include <stdio.h>
+#include <stdlib.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <time.h>
+#include <string.h>
+
+struct avro_file_reader_t {
+	avro_schema_t writers_schema;
+	avro_reader_t reader;
+	char sync[16];
+	int64_t blocks_read;
+	int64_t blocks_total;
+	int64_t current_blocklen;
+};
+
+struct avro_file_writer_t {
+	avro_schema_t writers_schema;
+	avro_writer_t writer;
+	char sync[16];
+	int block_count;
+	avro_writer_t datum_writer;
+	char datum_buffer[16 * 1024];
+};
+
+/* TODO: should we just read /dev/random? */
+static void generate_sync(struct avro_file_writer_t *w)
+{
+	int i;
+	srand(time(NULL));
+	for (i = 0; i < sizeof(w->sync); i++) {
+		w->sync[i] = ((double)rand() / (RAND_MAX + 1.0)) * 255;
+	}
+}
+
+static int write_sync(struct avro_file_writer_t *w)
+{
+	return avro_write(w->writer, w->sync, sizeof(w->sync));
+}
+
+static int write_header(struct avro_file_writer_t *w)
+{
+	int rval;
+	uint8_t version = 1;
+	/* TODO: remove this static buffer */
+	avro_writer_t schema_writer;
+	char schema_buf[64 * 1024];
+	const avro_encoding_t *enc = &avro_binary_encoding;
+
+	/* Generate random sync */
+	generate_sync(w);
+
+	check(rval, avro_write(w->writer, "Obj", 3));
+	check(rval, avro_write(w->writer, &version, 1));
+
+	check(rval, enc->write_long(w->writer, 3));
+	check(rval, enc->write_string(w->writer, "avro.sync"));
+	check(rval, enc->write_bytes(w->writer, w->sync, sizeof(w->sync)));
+	check(rval, enc->write_string(w->writer, "avro.codec"));
+	check(rval, enc->write_bytes(w->writer, "null", 4));
+	check(rval, enc->write_string(w->writer, "avro.schema"));
+	schema_writer = avro_writer_memory(schema_buf, sizeof(schema_buf));
+	rval = avro_schema_to_json(w->writers_schema, schema_writer);
+	if (rval) {
+		avro_writer_free(schema_writer);
+		return rval;
+	}
+	check(rval,
+	      enc->write_bytes(w->writer, schema_buf,
+			       avro_writer_tell(schema_writer)));
+	check(rval, enc->write_long(w->writer, 0));
+	return write_sync(w);
+}
+
+static int
+file_writer_init_fp(const char *path, const char *mode,
+		    struct avro_file_writer_t *w)
+{
+	FILE *fp = fopen(path, mode);
+	if (!fp) {
+		return ENOMEM;
+	}
+	w->writer = avro_writer_file(fp);
+	if (!w->writer) {
+		return ENOMEM;
+	}
+	return 0;
+}
+
+static int
+file_writer_create(const char *path, avro_schema_t schema,
+		   struct avro_file_writer_t *w)
+{
+	int rval = file_writer_init_fp(path, "wx", w);
+	if (rval) {
+		check(rval, file_writer_init_fp(path, "w", w));
+	}
+
+	w->datum_writer =
+	    avro_writer_memory(w->datum_buffer, sizeof(w->datum_buffer));
+	if (!w->datum_writer) {
+		avro_writer_free(w->writer);
+		return ENOMEM;
+	}
+
+	w->writers_schema = schema;
+	return write_header(w);
+}
+
+int
+avro_file_writer_create(const char *path, avro_schema_t schema,
+			avro_file_writer_t * writer)
+{
+	struct avro_file_writer_t *w;
+	int rval;
+	if (!path || !is_avro_schema(schema) || !writer) {
+		return EINVAL;
+	}
+	w = malloc(sizeof(struct avro_file_writer_t));
+	if (!w) {
+		return ENOMEM;
+	}
+	rval = file_writer_create(path, schema, w);
+	if (rval) {
+		free(w);
+		return rval;
+	}
+	*writer = w;
+	return 0;
+}
+
+static int file_read_header(avro_reader_t reader,
+			    avro_schema_t * writers_schema, char *sync,
+			    int synclen)
+{
+	int rval;
+	avro_schema_t meta_schema;
+	avro_schema_t meta_values_schema;
+	avro_datum_t meta;
+	char magic[4];
+	avro_datum_t schema_bytes;
+	char *p;
+	int64_t len;
+	avro_schema_error_t schema_error;
+
+	check(rval, avro_read(reader, magic, sizeof(magic)));
+	if (magic[0] != 'O' || magic[1] != 'b' || magic[2] != 'j'
+	    || magic[3] != 1) {
+		return EILSEQ;
+	}
+	meta_values_schema = avro_schema_bytes();
+	meta_schema = avro_schema_map(meta_values_schema);
+	rval = avro_read_data(reader, meta_schema, NULL, &meta);
+	if (rval) {
+		return EILSEQ;
+	}
+	check(rval, avro_map_get(meta, "avro.schema", &schema_bytes));
+	avro_bytes_get(schema_bytes, &p, &len);
+	check(rval,
+	      avro_schema_from_json(p, len, writers_schema, &schema_error));
+	avro_schema_decref(meta);
+	return avro_read(reader, sync, synclen);
+}
+
+static int file_writer_open(const char *path, struct avro_file_writer_t *w)
+{
+	int rval;
+	FILE *fp;
+	avro_reader_t reader;
+
+	fp = fopen(path, "r");
+	if (!fp) {
+		return errno;
+	}
+	reader = avro_reader_file(fp);
+	if (!reader) {
+		return ENOMEM;
+	}
+	rval =
+	    file_read_header(reader, &w->writers_schema, w->sync,
+			     sizeof(w->sync));
+	avro_reader_free(reader);
+	/* Position to end of file and get ready to write */
+	rval = file_writer_init_fp(path, "a", w);
+	if (rval) {
+		free(w);
+	}
+	return rval;
+}
+
+int avro_file_writer_open(const char *path, avro_file_writer_t * writer)
+{
+	struct avro_file_writer_t *w;
+	int rval;
+	if (!path || !writer) {
+		return EINVAL;
+	}
+	w = malloc(sizeof(struct avro_file_writer_t));
+	if (!w) {
+		return ENOMEM;
+	}
+	rval = file_writer_open(path, w);
+	if (rval) {
+		free(w);
+		return rval;
+	}
+
+	*writer = w;
+	return 0;
+}
+
+static int file_read_block_count(avro_file_reader_t r)
+{
+	int rval;
+	const avro_encoding_t *enc = &avro_binary_encoding;
+	check(rval, enc->read_long(r->reader, &r->blocks_total));
+	check(rval, enc->read_long(r->reader, &r->current_blocklen));
+	r->blocks_read = 0;
+	return 0;
+}
+
+int avro_file_reader(const char *path, avro_file_reader_t * reader)
+{
+	int rval;
+	FILE *fp;
+	struct avro_file_reader_t *r =
+	    malloc(sizeof(struct avro_file_reader_t));
+	if (!r) {
+		return ENOMEM;
+	}
+
+	fp = fopen(path, "r");
+	if (!fp) {
+		return errno;
+	}
+	r->reader = avro_reader_file(fp);
+	if (!r->reader) {
+		return ENOMEM;
+	}
+	rval = file_read_header(r->reader, &r->writers_schema, r->sync,
+				sizeof(r->sync));
+	if (rval == 0) {
+		rval = file_read_block_count(r);
+		if (rval == 0) {
+			*reader = r;
+		}
+	}
+	return rval;
+}
+
+static int file_write_block(avro_file_writer_t w)
+{
+	const avro_encoding_t *enc = &avro_binary_encoding;
+	int rval;
+
+	if (w->block_count) {
+		int64_t blocklen = avro_writer_tell(w->datum_writer);
+		/* Write the block count */
+		check(rval, enc->write_long(w->writer, w->block_count));
+		/* Write the block length */
+		check(rval, enc->write_long(w->writer, blocklen));
+		/* Write the block */
+		check(rval, avro_write(w->writer, w->datum_buffer, blocklen));
+		/* Write the sync marker */
+		check(rval, write_sync(w));
+		/* Reset the datum writer */
+		avro_writer_reset(w->writer);
+		w->block_count = 0;
+	}
+	return 0;
+}
+
+int avro_file_writer_append(avro_file_writer_t w, avro_datum_t datum)
+{
+	int rval;
+	if (!w || !datum) {
+		return EINVAL;
+	}
+	rval = avro_write_data(w->datum_writer, w->writers_schema, datum);
+	if (rval) {
+		check(rval, file_write_block(w));
+		rval =
+		    avro_write_data(w->datum_writer, w->writers_schema, datum);
+		if (rval) {
+			/* TODO: if the datum encoder larger than our buffer,
+			   just write a single large datum */
+			return rval;
+		}
+	}
+	w->block_count++;
+	return 0;
+}
+
+int avro_file_writer_sync(avro_file_writer_t w)
+{
+	return file_write_block(w);
+}
+
+int avro_file_writer_flush(avro_file_writer_t w)
+{
+	int rval;
+	check(rval, file_write_block(w));
+	avro_writer_flush(w->writer);
+	return 0;
+}
+
+int avro_file_writer_close(avro_file_writer_t w)
+{
+	int rval;
+	check(rval, avro_file_writer_flush(w));
+	avro_writer_free(w->writer);
+	return 0;
+}
+
+int avro_file_reader_read(avro_file_reader_t r, avro_schema_t readers_schema,
+			  avro_datum_t * datum)
+{
+	int rval;
+	char sync[16];
+
+	if (!r || !datum) {
+		return EINVAL;
+	}
+
+	check(rval,
+	      avro_read_data(r->reader, r->writers_schema, readers_schema,
+			     datum));
+	r->blocks_read++;
+
+	if (r->blocks_read == r->blocks_total) {
+		check(rval, avro_read(r->reader, sync, sizeof(sync)));
+		if (memcmp(r->sync, sync, sizeof(r->sync)) != 0) {
+			/* wrong sync bytes */
+			return EILSEQ;
+		}
+		/* For now, ignore errors (e.g. EOF) */
+		file_read_block_count(r);
+	}
+	return 0;
+}
+
+int avro_file_reader_close(avro_file_reader_t reader)
+{
+	avro_reader_free(reader->reader);
+	return 0;
+}

Modified: hadoop/avro/trunk/lang/c/src/io.c
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/src/io.c?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/src/io.c (original)
+++ hadoop/avro/trunk/lang/c/src/io.c Sat Feb  6 00:56:51 2010
@@ -250,6 +250,28 @@
 	return EINVAL;
 }
 
+void avro_writer_reset(avro_writer_t writer)
+{
+	if (is_memory_io(writer)) {
+		avro_writer_to_memory(writer)->written = 0;
+	}
+}
+
+int64_t avro_writer_tell(avro_writer_t writer)
+{
+	if (is_memory_io(writer)) {
+		return avro_writer_to_memory(writer)->written;
+	}
+	return -1;
+}
+
+void avro_writer_flush(avro_writer_t writer)
+{
+	if (is_file_io(writer)) {
+		fflush(avro_writer_to_file(writer)->fp);
+	}
+}
+
 void avro_writer_dump(avro_writer_t writer, FILE * fp)
 {
 	if (is_memory_io(writer)) {
@@ -271,6 +293,7 @@
 	if (is_memory_io(reader)) {
 		free(avro_reader_to_memory(reader));
 	} else if (is_file_io(reader)) {
+		fclose(avro_reader_to_file(reader)->fp);
 		free(avro_reader_to_file(reader));
 	}
 }
@@ -280,6 +303,7 @@
 	if (is_memory_io(writer)) {
 		free(avro_writer_to_memory(writer));
 	} else if (is_file_io(writer)) {
+		fclose(avro_writer_to_file(writer)->fp);
 		free(avro_writer_to_file(writer));
 	}
 }

Modified: hadoop/avro/trunk/lang/c/tests/Makefile.am
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/tests/Makefile.am?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/tests/Makefile.am (original)
+++ hadoop/avro/trunk/lang/c/tests/Makefile.am Sat Feb  6 00:56:51 2010
@@ -7,6 +7,8 @@
 
 check_PROGRAMS=test_avro_schema test_avro_data
 
+noinst_PROGRAMS=generate_interop_data test_interop_data
+
 test_LDADD=$(top_builddir)/src/libavro.la
 
 test_avro_schema_SOURCES=test_avro_schema.c
@@ -15,4 +17,10 @@
 test_avro_data_SOURCES=test_avro_data.c
 test_avro_data_LDADD=$(test_LDADD)
 
+generate_interop_data_SOURCES=generate_interop_data.c
+generate_interop_data_LDADD=$(test_LDADD)
+
+test_interop_data_SOURCES=test_interop_data.c
+test_interop_data_LDADD=$(test_LDADD)
+
 TESTS=$(check_PROGRAMS) test_valgrind

Added: hadoop/avro/trunk/lang/c/tests/generate_interop_data.c
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/tests/generate_interop_data.c?rev=907144&view=auto
==============================================================================
--- hadoop/avro/trunk/lang/c/tests/generate_interop_data.c (added)
+++ hadoop/avro/trunk/lang/c/tests/generate_interop_data.c Sat Feb  6 00:56:51 2010
@@ -0,0 +1,79 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <avro.h>
+#include "avro_private.h"
+
+int main(int argc, char *argv[])
+{
+	int rval;
+	avro_file_writer_t file_writer;
+	avro_file_reader_t file_reader;
+	char outpath[128];
+	FILE *fp;
+	char jsontext[16 * 1024];
+	avro_schema_t schema;
+	avro_schema_error_t schema_error;
+	avro_datum_t interop;
+	avro_datum_t array_datum;
+	avro_datum_t node_datum;
+	avro_datum_t out_datum;
+
+	if (argc != 3) {
+		exit(EXIT_FAILURE);
+	}
+	snprintf(outpath, sizeof(outpath), "%s/c.avro", argv[2]);
+	fprintf(stderr, "Writing %s\n", outpath);
+
+	fp = fopen(argv[1], "r");
+	rval = fread(jsontext, 1, sizeof(jsontext) - 1, fp);
+	jsontext[rval] = '\0';
+
+	check(rval,
+	      avro_schema_from_json(jsontext, rval, &schema, &schema_error));
+	check(rval, avro_file_writer_create(outpath, schema, &file_writer));
+
+	/* TODO: create a method for generating random data from schema */
+	interop = avro_record("interop");
+	avro_record_set(interop, "intField", avro_int32(42));
+	avro_record_set(interop, "longField", avro_int64(4242));
+	avro_record_set(interop, "stringField",
+			avro_wrapstring("Follow your bliss."));
+	avro_record_set(interop, "boolField", avro_boolean(1));
+	avro_record_set(interop, "floatField", avro_float(3.14159265));
+	avro_record_set(interop, "doubleField", avro_double(2.71828183));
+	avro_record_set(interop, "bytesField", avro_bytes("abcd", 4));
+	avro_record_set(interop, "nullField", avro_null());
+
+	array_datum = avro_array();
+	avro_array_append_datum(array_datum, avro_double(1.0));
+	avro_array_append_datum(array_datum, avro_double(2.0));
+	avro_array_append_datum(array_datum, avro_double(3.0));
+	avro_record_set(interop, "arrayField", array_datum);
+
+	avro_record_set(interop, "mapField", avro_map());
+	avro_record_set(interop, "unionField", avro_double(1.61803399));
+	avro_record_set(interop, "enumField", avro_enum("Kind", "B"));
+	avro_record_set(interop, "fixedField",
+			avro_fixed("MD5", "1234567890123456", 16));
+
+	node_datum = avro_record("Node");
+	avro_record_set(node_datum, "label",
+			avro_wrapstring("If you label me, you negate me."));
+	avro_record_set(node_datum, "children", avro_array());
+	avro_record_set(interop, "recordField", node_datum);
+
+	rval = avro_file_writer_append(file_writer, interop);
+	if (rval) {
+		fprintf(stderr, "Unable to append data to interop file!\n");
+	}
+	check(rval, avro_file_writer_close(file_writer));
+
+	check(rval, avro_file_reader(outpath, &file_reader));
+	check(rval, avro_file_reader_read(file_reader, NULL, &out_datum));
+	if (!avro_datum_equal(interop, out_datum)) {
+		exit(EXIT_FAILURE);
+	}
+	check(rval, avro_file_reader_close(file_reader));
+	return 0;
+}

Added: hadoop/avro/trunk/lang/c/tests/test_interop_data.c
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/tests/test_interop_data.c?rev=907144&view=auto
==============================================================================
--- hadoop/avro/trunk/lang/c/tests/test_interop_data.c (added)
+++ hadoop/avro/trunk/lang/c/tests/test_interop_data.c Sat Feb  6 00:56:51 2010
@@ -0,0 +1,9 @@
+#include <stdio.h>
+#include <unistd.h>
+#include <avro.h>
+#include "avro_private.h"
+
+int main(void)
+{
+	return 0;
+}

Modified: hadoop/avro/trunk/lang/c/version.sh
URL: http://svn.apache.org/viewvc/hadoop/avro/trunk/lang/c/version.sh?rev=907144&r1=907143&r2=907144&view=diff
==============================================================================
--- hadoop/avro/trunk/lang/c/version.sh (original)
+++ hadoop/avro/trunk/lang/c/version.sh Sat Feb  6 00:56:51 2010
@@ -18,9 +18,9 @@
 #         libavro_binary_age = 0
 #         libavro_interface_age = 0
 #
-libavro_micro_version=15
+libavro_micro_version=16
 libavro_interface_age=0
-libavro_binary_age=0
+libavro_binary_age=1
 
 # IGNORE EVERYTHING ELSE FROM HERE DOWN.........
 if test $# != 1; then