You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by bu...@apache.org on 2017/02/02 23:00:19 UTC

avro git commit: AVRO-1642: Java: Do not generate invalid all-args constructor. Contributed by Prateek Rungta and Barry Jones.

Repository: avro
Updated Branches:
  refs/heads/branch-1.7 3345e1323 -> 646dee5e9


AVRO-1642: Java: Do not generate invalid all-args constructor. Contributed by Prateek Rungta and Barry Jones.

Backport:
Amending-Author: Ben McCann <bm...@linkedin.com>


Project: http://git-wip-us.apache.org/repos/asf/avro/repo
Commit: http://git-wip-us.apache.org/repos/asf/avro/commit/646dee5e
Tree: http://git-wip-us.apache.org/repos/asf/avro/tree/646dee5e
Diff: http://git-wip-us.apache.org/repos/asf/avro/diff/646dee5e

Branch: refs/heads/branch-1.7
Commit: 646dee5e9196ff0784776b55eba9d9ed9ba2a805
Parents: 3345e13
Author: Al B <al...@ACAC04CC.ipt.aol.com>
Authored: Mon Jan 30 13:56:53 2017 -0800
Committer: Sean Busbey <bu...@apache.org>
Committed: Thu Feb 2 12:56:02 2017 -0600

----------------------------------------------------------------------
 .../compiler/specific/SpecificCompiler.java     |  57 ++++
 .../specific/templates/java/classic/record.vm   |  10 +-
 .../avro/compiler/TestSpecificCompiler.java     | 216 -------------
 .../compiler/specific/TestSpecificCompiler.java | 312 +++++++++++++++++++
 4 files changed, 378 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/avro/blob/646dee5e/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java b/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
index 00be382..af4ffda 100644
--- a/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
+++ b/lang/java/compiler/src/main/java/org/apache/avro/compiler/specific/SpecificCompiler.java
@@ -58,6 +58,27 @@ import org.slf4j.LoggerFactory;
  * Java reserved keywords are mangled to preserve compilation.
  */
 public class SpecificCompiler {
+
+  /*
+   * From Section 4.10 of the Java VM Specification:
+   *    A method descriptor is valid only if it represents method parameters with a total length of 255 or less,
+   *    where that length includes the contribution for this in the case of instance or interface method invocations.
+   *    The total length is calculated by summing the contributions of the individual parameters, where a parameter
+   *    of type long or double contributes two units to the length and a parameter of any other type contributes one unit.
+   *
+   * Arguments of type Double/Float contribute 2 "parameter units" to this limit, all other types contribute 1
+   * "parameter unit". All instance methods for a class are passed a reference to the instance (`this), and hence,
+   * they are permitted at most `JVM_METHOD_ARG_LIMIT-1` "parameter units" for their arguments.
+   *
+   * @see <a href="http://docs.oracle.com/javase/specs/jvms/se7/html/jvms-4.html#jvms-4.10">JVM Spec: Section 4.10</a>
+   */
+  private static final int JVM_METHOD_ARG_LIMIT = 255;
+
+  /*
+   * Note: This is protected instead of private only so it's visible for testing.
+   */
+  protected static final int MAX_FIELD_PARAMETER_UNIT_COUNT = JVM_METHOD_ARG_LIMIT - 1;
+
   public static enum FieldVisibility {
     PUBLIC, PUBLIC_DEPRECATED, PRIVATE
   }
@@ -68,8 +89,16 @@ public class SpecificCompiler {
   private String templateDir;
   private FieldVisibility fieldVisibility = FieldVisibility.PUBLIC_DEPRECATED;
   private boolean createSetters = true;
+  private boolean createAllArgsConstructor = true;
   private String outputCharacterEncoding;
 
+  /*
+   * Used in the record.vm template.
+   */
+  public boolean isCreateAllArgsConstructor() {
+    return createAllArgsConstructor;
+  }
+
   /* List of Java reserved words from
    * http://java.sun.com/docs/books/jls/third_edition/html/lexical.html. */
   private static final Set<String> RESERVED_WORDS = new HashSet<String>(
@@ -368,6 +397,33 @@ public class SpecificCompiler {
     }
   }
 
+  /**
+   * Returns the number of parameter units required by fields for the
+   * AllArgsConstructor.
+   *
+   * @param record a Record schema
+   */
+  protected int calcAllArgConstructorParameterUnits(Schema record) {
+
+    if (record.getType() != Schema.Type.RECORD)
+      throw new RuntimeException("This method must only be called for record schemas.");
+
+    return record.getFields().size();
+  }
+
+  protected void validateRecordForCompilation(Schema record) {
+    this.createAllArgsConstructor =
+        calcAllArgConstructorParameterUnits(record) <= MAX_FIELD_PARAMETER_UNIT_COUNT;
+
+    if (!this.createAllArgsConstructor)
+      new Slf4jLogChute().log(LogChute.WARN_ID, "Record '" + record.getFullName() +
+              "' contains more than " + MAX_FIELD_PARAMETER_UNIT_COUNT +
+              " parameters which exceeds the JVM " +
+              "spec for the number of permitted constructor arguments. Clients must " +
+              "rely on the builder pattern to create objects instead. For more info " +
+              "see JIRA ticket AVRO-1642.");
+  }
+
   OutputFile compile(Schema schema) {
     schema = addStringType(schema);               // annotate schema as needed
     String output = "";
@@ -377,6 +433,7 @@ public class SpecificCompiler {
 
     switch (schema.getType()) {
     case RECORD:
+      validateRecordForCompilation(schema);
       output = renderTemplate(templateDir+"record.vm", context);
       break;
     case ENUM:

http://git-wip-us.apache.org/repos/asf/avro/blob/646dee5e/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm b/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
index 1becde0..8a3739d 100644
--- a/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
+++ b/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
@@ -65,7 +65,7 @@ public class ${this.mangle($schema.getName())}#if ($schema.isError()) extends or
    * one should use <code>newBuilder()</code>. 
    */
   public ${this.mangle($schema.getName())}() {}
-
+#if ($this.isCreateAllArgsConstructor())
   /**
    * All-args constructor.
 #foreach ($field in $schema.getFields())
@@ -78,6 +78,14 @@ public class ${this.mangle($schema.getName())}#if ($schema.isError()) extends or
     this.${this.mangle($field.name())} = ${this.mangle($field.name())};
 #end
   }
+#else
+  /**
+   * This schema contains more than 254 fields which exceeds the maximum number
+   * of permitted constructor parameters in the JVM. An all-args constructor
+   * will not be generated. Please use <code>newBuilder()</code> to instantiate
+   * objects instead.
+   */
+#end
 #end
 
 #end

http://git-wip-us.apache.org/repos/asf/avro/blob/646dee5e/lang/java/compiler/src/test/java/org/apache/avro/compiler/TestSpecificCompiler.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/test/java/org/apache/avro/compiler/TestSpecificCompiler.java b/lang/java/compiler/src/test/java/org/apache/avro/compiler/TestSpecificCompiler.java
deleted file mode 100644
index 35c1936..0000000
--- a/lang/java/compiler/src/test/java/org/apache/avro/compiler/TestSpecificCompiler.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/**
- * 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.avro.compiler;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertThat;
-import static org.hamcrest.CoreMatchers.not;
-import static org.hamcrest.CoreMatchers.equalTo;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.nio.charset.Charset;
-
-import org.apache.avro.AvroTestUtil;
-import org.apache.avro.Schema;
-import org.apache.avro.compiler.specific.SpecificCompiler;
-import org.apache.avro.generic.GenericData.StringType;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-@RunWith(JUnit4.class)
-public class TestSpecificCompiler {
-  private final String schemaSrcPath = "src/test/resources/simple_record.avsc";
-  private final String velocityTemplateDir =
-      "src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/";
-  private File src;
-  private File outputDir;
-  private File outputFile;
-
-  @Before
-  public void setUp() {
-    this.src = new File(this.schemaSrcPath);
-    this.outputDir = AvroTestUtil.tempDirectory(getClass(), "specific-output");
-    this.outputFile = new File(this.outputDir, "SimpleRecord.java");
-    if (outputFile.exists() && !outputFile.delete()) {
-      throw new IllegalStateException("unable to delete " + outputFile);
-    }
-  }
-
-  @After
-  public void tearDow() {
-    if (this.outputFile != null) {
-      this.outputFile.delete();
-    }
-  }
-
-  private SpecificCompiler createCompiler() throws IOException {
-    Schema.Parser parser = new Schema.Parser();
-    Schema schema = parser.parse(this.src);
-    SpecificCompiler compiler = new SpecificCompiler(schema);
-    compiler.setTemplateDir(this.velocityTemplateDir);
-    compiler.setStringType(StringType.CharSequence);
-    return compiler;
-  }
-
-  @Test
-  public void testCanReadTemplateFilesOnTheFilesystem() throws IOException, URISyntaxException{
-    SpecificCompiler compiler = createCompiler();
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-  }
-
-  @Test
-  public void testPublicFieldVisibility() throws IOException {
-    SpecificCompiler compiler = createCompiler();
-    compiler.setFieldVisibility(SpecificCompiler.FieldVisibility.PUBLIC);
-    assertFalse(compiler.deprecatedFields());
-    assertTrue(compiler.publicFields());
-    assertFalse(compiler.privateFields());
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
-    String line = null;
-    while ((line = reader.readLine()) != null) {
-      // No line, once trimmed, should start with a deprecated field declaration
-      // nor a private field declaration.  Since the nested builder uses private
-      // fields, we cannot do the second check.
-      line = line.trim();
-      assertFalse("Line started with a deprecated field declaration: " + line,
-        line.startsWith("@Deprecated public int value"));
-    }
-    reader.close();
-  }
-
-  @Test
-  public void testPublicDeprecatedFieldVisibility() throws IOException {
-    SpecificCompiler compiler = createCompiler();
-    assertTrue(compiler.deprecatedFields());
-    assertTrue(compiler.publicFields());
-    assertFalse(compiler.privateFields());
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
-    String line = null;
-    while ((line = reader.readLine()) != null) {
-      // No line, once trimmed, should start with a public field declaration
-      line = line.trim();
-      assertFalse("Line started with a public field declaration: " + line,
-        line.startsWith("public int value"));
-    }
-    reader.close();
-  }
-
-  @Test
-  public void testPrivateFieldVisibility() throws IOException {
-    SpecificCompiler compiler = createCompiler();
-    compiler.setFieldVisibility(SpecificCompiler.FieldVisibility.PRIVATE);
-    assertFalse(compiler.deprecatedFields());
-    assertFalse(compiler.publicFields());
-    assertTrue(compiler.privateFields());
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
-    String line = null;
-    while ((line = reader.readLine()) != null) {
-      // No line, once trimmed, should start with a public field declaration
-      // or with a deprecated public field declaration
-      line = line.trim();
-      assertFalse("Line started with a public field declaration: " + line,
-        line.startsWith("public int value"));
-      assertFalse("Line started with a deprecated field declaration: " + line,
-        line.startsWith("@Deprecated public int value"));
-    }
-    reader.close();
-  }
-
-  @Test
-  public void testSettersCreatedByDefault() throws IOException {
-    SpecificCompiler compiler = createCompiler();
-    assertTrue(compiler.isCreateSetters());
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
-    int foundSetters = 0;
-    String line = null;
-    while ((line = reader.readLine()) != null) {
-      // We should find the setter in the main class
-      line = line.trim();
-      if (line.startsWith("public void setValue(")) {
-        foundSetters++;
-      }
-    }
-    reader.close();
-    assertEquals("Found the wrong number of setters", 1, foundSetters);
-  }
-
-  @Test
-  public void testSettersNotCreatedWhenOptionTurnedOff() throws IOException {
-    SpecificCompiler compiler = createCompiler();
-    compiler.setCreateSetters(false);
-    assertFalse(compiler.isCreateSetters());
-    compiler.compileToDestination(this.src, this.outputDir);
-    assertTrue(this.outputFile.exists());
-    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
-    String line = null;
-    while ((line = reader.readLine()) != null) {
-      // No setter should be found
-      line = line.trim();
-      assertFalse("No line should include the setter: " + line,
-        line.startsWith("public void setValue("));
-    }
-    reader.close();
-  }
-
-  @Test
-  public void testSettingOutputCharacterEncoding() throws Exception {
-    SpecificCompiler compiler = createCompiler();
-    // Generated file in default encoding
-    compiler.compileToDestination(this.src, this.outputDir);
-    byte[] fileInDefaultEncoding = new byte[(int) this.outputFile.length()];
-    FileInputStream is = new FileInputStream(this.outputFile);
-    is.read(fileInDefaultEncoding);
-    is.close(); //close input stream otherwise delete might fail
-    if (!this.outputFile.delete()) {
-      throw new IllegalStateException("unable to delete " + this.outputFile); //delete otherwise compiler might not overwrite because src timestamp hasnt changed.
-    }
-    // Generate file in another encoding (make sure it has different number of bytes per character)
-    String differentEncoding = Charset.defaultCharset().equals(Charset.forName("UTF-16")) ? "UTF-32" : "UTF-16";
-    compiler.setOutputCharacterEncoding(differentEncoding);
-    compiler.compileToDestination(this.src, this.outputDir);
-    byte[] fileInDifferentEncoding = new byte[(int) this.outputFile.length()];
-    is = new FileInputStream(this.outputFile);
-    is.read(fileInDifferentEncoding);
-    is.close();
-    // Compare as bytes
-    assertThat("Generated file should contain different bytes after setting non-default encoding",
-      fileInDefaultEncoding, not(equalTo(fileInDifferentEncoding)));
-    // Compare as strings
-    assertThat("Generated files should contain the same characters in the proper encodings",
-      new String(fileInDefaultEncoding), equalTo(new String(fileInDifferentEncoding, differentEncoding)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/avro/blob/646dee5e/lang/java/compiler/src/test/java/org/apache/avro/compiler/specific/TestSpecificCompiler.java
----------------------------------------------------------------------
diff --git a/lang/java/compiler/src/test/java/org/apache/avro/compiler/specific/TestSpecificCompiler.java b/lang/java/compiler/src/test/java/org/apache/avro/compiler/specific/TestSpecificCompiler.java
new file mode 100644
index 0000000..a1118ba
--- /dev/null
+++ b/lang/java/compiler/src/test/java/org/apache/avro/compiler/specific/TestSpecificCompiler.java
@@ -0,0 +1,312 @@
+/**
+ * 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.avro.compiler.specific;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertThat;
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.CoreMatchers.equalTo;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.avro.AvroTestUtil;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.generic.GenericData.StringType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import javax.tools.JavaCompiler;
+import javax.tools.StandardJavaFileManager;
+import javax.tools.ToolProvider;
+
+@RunWith(JUnit4.class)
+public class TestSpecificCompiler {
+  private final String schemaSrcPath = "src/test/resources/simple_record.avsc";
+  private final String velocityTemplateDir =
+      "src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/";
+  private File src;
+  private File outputDir;
+  private File outputFile;
+
+  @Before
+  public void setUp() {
+    this.src = new File(this.schemaSrcPath);
+    this.outputDir = AvroTestUtil.tempDirectory(getClass(), "specific-output");
+    this.outputFile = new File(this.outputDir, "SimpleRecord.java");
+    if (outputFile.exists() && !outputFile.delete()) {
+      throw new IllegalStateException("unable to delete " + outputFile);
+    }
+  }
+
+  @After
+  public void tearDow() {
+    if (this.outputFile != null) {
+      this.outputFile.delete();
+    }
+  }
+
+  /** Uses the system's java compiler to actually compile the generated code. */
+  static void assertCompilesWithJavaCompiler(Collection<SpecificCompiler.OutputFile> outputs)
+          throws IOException {
+    if (outputs.isEmpty())
+      return;               // Nothing to compile!
+
+    JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+    StandardJavaFileManager fileManager =
+            compiler.getStandardFileManager(null, null, null);
+
+    File dstDir = AvroTestUtil.tempFile(TestSpecificCompiler.class, "realCompiler");
+    List<File> javaFiles = new ArrayList<File>();
+    for (SpecificCompiler.OutputFile o : outputs) {
+      javaFiles.add(o.writeToDestination(null, dstDir));
+    }
+
+    JavaCompiler.CompilationTask cTask = compiler.getTask(null, fileManager,
+            null, null, null, fileManager.getJavaFileObjects(
+                    javaFiles.toArray(new File[javaFiles.size()])));
+    boolean compilesWithoutError = cTask.call();
+    assertTrue(compilesWithoutError);
+  }
+
+  private static Schema createSampleRecordSchema(int numStringFields, int numDoubleFields) {
+    SchemaBuilder.FieldAssembler<Schema> sb = SchemaBuilder.record("sample.record").fields();
+    for (int i = 0; i < numStringFields; i++) {
+      sb.name("sf_" + i).type().stringType().noDefault();
+    }
+    for (int i = 0; i < numDoubleFields; i++) {
+      sb.name("df_" + i).type().doubleType().noDefault();
+    }
+    return sb.endRecord();
+  }
+
+  private SpecificCompiler createCompiler() throws IOException {
+    Schema.Parser parser = new Schema.Parser();
+    Schema schema = parser.parse(this.src);
+    SpecificCompiler compiler = new SpecificCompiler(schema);
+    compiler.setTemplateDir(this.velocityTemplateDir);
+    compiler.setStringType(StringType.CharSequence);
+    return compiler;
+  }
+
+  @Test
+  public void testCreateAllArgsConstructor() throws Exception {
+    SpecificCompiler compiler = createCompiler();
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    String line = null;
+    boolean foundAllArgsConstructor = false;
+    while (!foundAllArgsConstructor && (line = reader.readLine()) != null) {
+      foundAllArgsConstructor = line.contains("All-args constructor");
+    }
+    assertTrue(foundAllArgsConstructor);
+  }
+
+  @Test
+  public void testMaxValidParameterCounts() throws Exception {
+    Schema validSchema1 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT, 0);
+    assertCompilesWithJavaCompiler(new SpecificCompiler(validSchema1).compile());
+
+    Schema validSchema2 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT - 2, 1);
+    assertCompilesWithJavaCompiler(new SpecificCompiler(validSchema1).compile());
+  }
+
+  @Test
+  public void testInvalidParameterCounts() throws Exception {
+    Schema invalidSchema1 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT + 1, 0);
+    SpecificCompiler compiler = new SpecificCompiler(invalidSchema1);
+    assertCompilesWithJavaCompiler(compiler.compile());
+
+    Schema invalidSchema2 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT, 10);
+    compiler = new SpecificCompiler(invalidSchema2);
+    assertCompilesWithJavaCompiler(compiler.compile());
+  }
+
+  @Test
+  public void testMaxParameterCounts() throws Exception {
+    Schema validSchema1 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT, 0);
+    assertTrue(new SpecificCompiler(validSchema1).compile().size() > 0);
+
+    Schema validSchema2 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT - 2, 1);
+    assertTrue(new SpecificCompiler(validSchema2).compile().size() > 0);
+
+    Schema validSchema3 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT - 1, 1);
+    assertTrue(new SpecificCompiler(validSchema3).compile().size() > 0);
+
+    Schema validSchema4 = createSampleRecordSchema(SpecificCompiler.MAX_FIELD_PARAMETER_UNIT_COUNT + 1, 0);
+    assertTrue(new SpecificCompiler(validSchema4).compile().size() > 0);
+  }
+
+  @Test(expected=RuntimeException.class)
+  public void testCalcAllArgConstructorParameterUnitsFailure() {
+    Schema nonRecordSchema = SchemaBuilder.array().items().booleanType();
+    new SpecificCompiler().calcAllArgConstructorParameterUnits(nonRecordSchema);
+  }
+
+  @Test
+  public void testCanReadTemplateFilesOnTheFilesystem() throws IOException, URISyntaxException{
+    SpecificCompiler compiler = createCompiler();
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+  }
+
+  @Test
+  public void testPublicFieldVisibility() throws IOException {
+    SpecificCompiler compiler = createCompiler();
+    compiler.setFieldVisibility(SpecificCompiler.FieldVisibility.PUBLIC);
+    assertFalse(compiler.deprecatedFields());
+    assertTrue(compiler.publicFields());
+    assertFalse(compiler.privateFields());
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      // No line, once trimmed, should start with a deprecated field declaration
+      // nor a private field declaration.  Since the nested builder uses private
+      // fields, we cannot do the second check.
+      line = line.trim();
+      assertFalse("Line started with a deprecated field declaration: " + line,
+        line.startsWith("@Deprecated public int value"));
+    }
+    reader.close();
+  }
+
+  @Test
+  public void testPublicDeprecatedFieldVisibility() throws IOException {
+    SpecificCompiler compiler = createCompiler();
+    assertTrue(compiler.deprecatedFields());
+    assertTrue(compiler.publicFields());
+    assertFalse(compiler.privateFields());
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      // No line, once trimmed, should start with a public field declaration
+      line = line.trim();
+      assertFalse("Line started with a public field declaration: " + line,
+        line.startsWith("public int value"));
+    }
+    reader.close();
+  }
+
+  @Test
+  public void testPrivateFieldVisibility() throws IOException {
+    SpecificCompiler compiler = createCompiler();
+    compiler.setFieldVisibility(SpecificCompiler.FieldVisibility.PRIVATE);
+    assertFalse(compiler.deprecatedFields());
+    assertFalse(compiler.publicFields());
+    assertTrue(compiler.privateFields());
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      // No line, once trimmed, should start with a public field declaration
+      // or with a deprecated public field declaration
+      line = line.trim();
+      assertFalse("Line started with a public field declaration: " + line,
+        line.startsWith("public int value"));
+      assertFalse("Line started with a deprecated field declaration: " + line,
+        line.startsWith("@Deprecated public int value"));
+    }
+    reader.close();
+  }
+
+  @Test
+  public void testSettersCreatedByDefault() throws IOException {
+    SpecificCompiler compiler = createCompiler();
+    assertTrue(compiler.isCreateSetters());
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    int foundSetters = 0;
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      // We should find the setter in the main class
+      line = line.trim();
+      if (line.startsWith("public void setValue(")) {
+        foundSetters++;
+      }
+    }
+    reader.close();
+    assertEquals("Found the wrong number of setters", 1, foundSetters);
+  }
+
+  @Test
+  public void testSettersNotCreatedWhenOptionTurnedOff() throws IOException {
+    SpecificCompiler compiler = createCompiler();
+    compiler.setCreateSetters(false);
+    assertFalse(compiler.isCreateSetters());
+    compiler.compileToDestination(this.src, this.outputDir);
+    assertTrue(this.outputFile.exists());
+    BufferedReader reader = new BufferedReader(new FileReader(this.outputFile));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      // No setter should be found
+      line = line.trim();
+      assertFalse("No line should include the setter: " + line,
+        line.startsWith("public void setValue("));
+    }
+    reader.close();
+  }
+
+  @Test
+  public void testSettingOutputCharacterEncoding() throws Exception {
+    SpecificCompiler compiler = createCompiler();
+    // Generated file in default encoding
+    compiler.compileToDestination(this.src, this.outputDir);
+    byte[] fileInDefaultEncoding = new byte[(int) this.outputFile.length()];
+    FileInputStream is = new FileInputStream(this.outputFile);
+    is.read(fileInDefaultEncoding);
+    is.close(); //close input stream otherwise delete might fail
+    if (!this.outputFile.delete()) {
+      throw new IllegalStateException("unable to delete " + this.outputFile); //delete otherwise compiler might not overwrite because src timestamp hasnt changed.
+    }
+    // Generate file in another encoding (make sure it has different number of bytes per character)
+    String differentEncoding = Charset.defaultCharset().equals(Charset.forName("UTF-16")) ? "UTF-32" : "UTF-16";
+    compiler.setOutputCharacterEncoding(differentEncoding);
+    compiler.compileToDestination(this.src, this.outputDir);
+    byte[] fileInDifferentEncoding = new byte[(int) this.outputFile.length()];
+    is = new FileInputStream(this.outputFile);
+    is.read(fileInDifferentEncoding);
+    is.close();
+    // Compare as bytes
+    assertThat("Generated file should contain different bytes after setting non-default encoding",
+      fileInDefaultEncoding, not(equalTo(fileInDifferentEncoding)));
+    // Compare as strings
+    assertThat("Generated files should contain the same characters in the proper encodings",
+      new String(fileInDefaultEncoding), equalTo(new String(fileInDifferentEncoding, differentEncoding)));
+  }
+}