You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/11/29 21:25:00 UTC

[jira] [Commented] (AVRO-1904) C: record with no fields should be allowed

    [ https://issues.apache.org/jira/browse/AVRO-1904?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16703831#comment-16703831 ] 

ASF GitHub Bot commented on AVRO-1904:
--------------------------------------

dkulp closed pull request #159: AVRO-1904: C: Allow record to have no fields.
URL: https://github.com/apache/avro/pull/159
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/lang/c/src/avro_generic_internal.h b/lang/c/src/avro_generic_internal.h
index dec7652a0..dd6d52d32 100644
--- a/lang/c/src/avro_generic_internal.h
+++ b/lang/c/src/avro_generic_internal.h
@@ -60,7 +60,7 @@ typedef struct avro_generic_value_iface {
 
 
 #define avro_value_instance_size(gcls) \
-    ((gcls)->instance_size == NULL? 0: (gcls)->instance_size(&(gcls)->parent))
+    ((gcls)->instance_size == NULL ? (ssize_t)-1 : (ssize_t)(gcls)->instance_size(&(gcls)->parent))
 #define avro_value_init(gcls, self) \
     ((gcls)->init == NULL? EINVAL: (gcls)->init(&(gcls)->parent, (self)))
 #define avro_value_done(gcls, self) \
diff --git a/lang/c/src/datafile.c b/lang/c/src/datafile.c
index f2da60376..95096c1f3 100644
--- a/lang/c/src/datafile.c
+++ b/lang/c/src/datafile.c
@@ -460,11 +460,13 @@ static int file_read_block_count(avro_file_reader_t r)
 		r->current_blocklen = len;
 	}
 
-	check_prefix(rval, avro_read(r->reader, r->current_blockdata, len),
-		     "Cannot read file block: ");
+	if (len > 0) {
+		check_prefix(rval, avro_read(r->reader, r->current_blockdata, len),
+			     "Cannot read file block: ");
 
-	check_prefix(rval, avro_codec_decode(r->codec, r->current_blockdata, len),
-		     "Cannot decode file block: ");
+		check_prefix(rval, avro_codec_decode(r->codec, r->current_blockdata, len),
+			     "Cannot decode file block: ");
+	}
 
 	avro_reader_memory_set_source(r->block_reader, (const char *) r->codec->block_data, r->codec->used_size);
 
diff --git a/lang/c/src/generic.c b/lang/c/src/generic.c
index ddef81aaa..5f957b7db 100644
--- a/lang/c/src/generic.c
+++ b/lang/c/src/generic.c
@@ -551,16 +551,21 @@ avro_generic_link_init(const avro_value_iface_t *viface, void *vself)
 	    container_of(viface, avro_generic_link_value_iface_t, parent.parent);
 
 	avro_value_t  *self = (avro_value_t *) vself;
-	size_t  target_instance_size =
+	ssize_t  target_instance_size =
 	    avro_value_instance_size(iface->target_giface);
-	if (target_instance_size == 0) {
+	if (target_instance_size < 0) {
 		return EINVAL;
 	}
 
 	self->iface = &iface->target_giface->parent;
-	self->self = avro_malloc(target_instance_size);
-	if (self->self == NULL) {
-		return ENOMEM;
+
+	if (target_instance_size == 0) {
+		self->self = NULL;
+	} else {
+		self->self = avro_malloc(target_instance_size);
+		if (self->self == NULL) {
+			return ENOMEM;
+		}
 	}
 
 	rval = avro_value_init(iface->target_giface, self->self);
@@ -2124,8 +2129,8 @@ avro_generic_array_class(avro_schema_t schema, memoize_state_t *state)
 		return NULL;
 	}
 
-	size_t  child_size = avro_value_instance_size(child_giface);
-	if (child_size == 0) {
+	ssize_t  child_size = avro_value_instance_size(child_giface);
+	if (child_size < 0) {
 		avro_set_error("Array item class must provide instance_size");
 		avro_value_iface_decref(&child_giface->parent);
 		return NULL;
@@ -2780,8 +2785,8 @@ avro_generic_map_class(avro_schema_t schema, memoize_state_t *state)
 		return NULL;
 	}
 
-	size_t  child_size = avro_value_instance_size(child_giface);
-	if (child_size == 0) {
+	ssize_t  child_size = avro_value_instance_size(child_giface);
+	if (child_size < 0) {
 		avro_set_error("Map value class must provide instance_size");
 		avro_value_iface_decref(&child_giface->parent);
 		return NULL;
@@ -3103,14 +3108,19 @@ avro_generic_record_class(avro_schema_t schema, memoize_state_t *state)
 	size_t  field_ifaces_size =
 		sizeof(avro_generic_value_iface_t *) * iface->field_count;
 
-	iface->field_offsets = (size_t *) avro_malloc(field_offsets_size);
-	if (iface->field_offsets == NULL) {
-		goto error;
-	}
+	if (iface->field_count == 0) {
+		iface->field_offsets = NULL;
+		iface->field_ifaces = NULL;
+	} else {
+		iface->field_offsets = (size_t *) avro_malloc(field_offsets_size);
+		if (iface->field_offsets == NULL) {
+			goto error;
+		}
 
-	iface->field_ifaces = (avro_generic_value_iface_t **) avro_malloc(field_ifaces_size);
-	if (iface->field_ifaces == NULL) {
-		goto error;
+		iface->field_ifaces = (avro_generic_value_iface_t **) avro_malloc(field_ifaces_size);
+		if (iface->field_ifaces == NULL) {
+			goto error;
+		}
 	}
 
 	size_t  next_offset = sizeof(avro_generic_record_t);
@@ -3139,9 +3149,9 @@ avro_generic_record_class(avro_schema_t schema, memoize_state_t *state)
 			goto error;
 		}
 
-		size_t  field_size =
+		ssize_t  field_size =
 		    avro_value_instance_size(iface->field_ifaces[i]);
-		if (field_size == 0) {
+		if (field_size < 0) {
 			avro_set_error("Record field class must provide instance_size");
 			goto error;
 		}
@@ -3506,9 +3516,9 @@ avro_generic_union_class(avro_schema_t schema, memoize_state_t *state)
 			goto error;
 		}
 
-		size_t  branch_size =
+		ssize_t  branch_size =
 		    avro_value_instance_size(iface->branch_ifaces[i]);
-		if (branch_size == 0) {
+		if (branch_size < 0) {
 			avro_set_error("Union branch class must provide instance_size");
 			goto error;
 		}
@@ -3518,8 +3528,8 @@ avro_generic_union_class(avro_schema_t schema, memoize_state_t *state)
 			i, branch_size);
 #endif
 
-		if (branch_size > max_branch_size) {
-			max_branch_size = branch_size;
+		if ((size_t)branch_size > max_branch_size) {
+			max_branch_size = (size_t)branch_size;
 		}
 	}
 
diff --git a/lang/c/src/schema.c b/lang/c/src/schema.c
index 3ade1140e..d523d5569 100644
--- a/lang/c/src/schema.c
+++ b/lang/c/src/schema.c
@@ -919,10 +919,6 @@ avro_schema_from_json_t(json_t *json, avro_schema_t *schema,
 				return EINVAL;
 			}
 			num_fields = json_array_size(json_fields);
-			if (num_fields == 0) {
-				avro_set_error("Record type must have at least one field");
-				return EINVAL;
-			}
 			fullname = json_string_value(json_name);
 			if (!fullname) {
 				avro_set_error("Record type must have a \"name\"");
diff --git a/lang/c/tests/CMakeLists.txt b/lang/c/tests/CMakeLists.txt
index 445e689a7..c66f67c68 100644
--- a/lang/c/tests/CMakeLists.txt
+++ b/lang/c/tests/CMakeLists.txt
@@ -64,3 +64,4 @@ add_avro_test(test_refcount)
 add_avro_test(test_cpp test_cpp.cpp)
 add_avro_test(test_avro_1379)
 add_avro_test(test_avro_1691)
+add_avro_test(test_avro_1904)
diff --git a/lang/c/tests/schema_tests/pass/record_no_fields b/lang/c/tests/schema_tests/pass/record_no_fields
new file mode 100644
index 000000000..142f45272
--- /dev/null
+++ b/lang/c/tests/schema_tests/pass/record_no_fields
@@ -0,0 +1 @@
+{"type": "record", "name": "R", "fields": []}
diff --git a/lang/c/tests/test_avro_1904.c b/lang/c/tests/test_avro_1904.c
new file mode 100644
index 000000000..ac20e20d1
--- /dev/null
+++ b/lang/c/tests/test_avro_1904.c
@@ -0,0 +1,130 @@
+/*
+ * 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 <stdio.h>
+#include <sys/stat.h>
+#include "avro.h"
+
+#define NUM_RECORDS 100
+
+static const char *filename = "avro_file.dat";
+
+static const char PERSON_SCHEMA[] =
+	"{"
+	"    \"type\":\"record\","
+	"    \"name\":\"Person\","
+	"    \"fields\": ["
+	"    ]"
+	"}";
+
+static int read_data() {
+	int rval;
+	int records_read = 0;
+
+	avro_file_reader_t reader;
+	avro_value_iface_t *iface;
+	avro_value_t value;
+
+	fprintf(stderr, "\nReading...\n");
+
+	rval = avro_file_reader(filename, &reader);
+
+	if (rval) {
+		fprintf(stderr, "Error: %s\n", avro_strerror());
+		return EXIT_FAILURE;
+	}
+
+	avro_schema_t schema = avro_file_reader_get_writer_schema(reader);
+
+	iface = avro_generic_class_from_schema(schema);
+	avro_generic_value_new(iface, &value);
+
+	while ((rval = avro_file_reader_read_value(reader, &value)) == 0) {
+		records_read++;
+		avro_value_reset(&value);
+	}
+
+	avro_value_decref(&value);
+	avro_value_iface_decref(iface);
+	avro_schema_decref(schema);
+	avro_file_reader_close(reader);
+
+	fprintf(stderr, "read %d records.\n", records_read);
+
+	if (rval != EOF || records_read != NUM_RECORDS) {
+		fprintf(stderr, "Error: %s\n", avro_strerror());
+		return EXIT_FAILURE;
+	}
+
+	return EXIT_SUCCESS;
+}
+
+static int write_data() {
+	int  i;
+	avro_schema_t schema;
+	avro_schema_error_t error;
+	avro_file_writer_t writer;
+	avro_value_iface_t *iface;
+	avro_value_t value;
+
+	fprintf(stderr, "\nWriting...\n");
+
+	if (avro_schema_from_json(PERSON_SCHEMA, 0, &schema, &error)) {
+		fprintf(stderr, "Unable to parse schema\n");
+		return EXIT_FAILURE;
+	}
+
+	iface = avro_generic_class_from_schema(schema);
+	avro_generic_value_new(iface, &value);
+
+	if (avro_file_writer_create(filename, schema, &writer)) {
+		fprintf(stderr, "There was an error creating file: %s\n", avro_strerror());
+		return EXIT_FAILURE;
+	}
+
+	for (i = 0; i < NUM_RECORDS; i++) {
+		if (avro_file_writer_append_value(writer, &value)) {
+			fprintf(stderr, "There was an error creating file: %s\n", avro_strerror());
+			return EXIT_FAILURE;
+		}
+	}
+
+	if (avro_file_writer_close(writer)) {
+		fprintf(stderr, "There was an error creating file: %s\n", avro_strerror());
+		return EXIT_FAILURE;
+	}
+
+	avro_value_iface_decref(iface);
+	avro_value_decref(&value);
+	avro_schema_decref(schema);
+
+	return EXIT_SUCCESS;
+}
+
+int main()
+{
+	int read_data_result;
+
+	if (write_data()) {
+		return EXIT_FAILURE;
+	}
+
+	read_data_result = read_data();
+	remove(filename);
+
+	return read_data_result;
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> C: record with no fields should be allowed
> ------------------------------------------
>
>                 Key: AVRO-1904
>                 URL: https://issues.apache.org/jira/browse/AVRO-1904
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: c
>            Reporter: Ben Walsh
>            Priority: Minor
>         Attachments: AVRO-1904.patch
>
>
> Avro-C should allow record with no fields.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)