Return-Path: X-Original-To: apmail-drill-commits-archive@www.apache.org Delivered-To: apmail-drill-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E4120187A4 for ; Wed, 2 Mar 2016 23:52:20 +0000 (UTC) Received: (qmail 93894 invoked by uid 500); 2 Mar 2016 23:52:20 -0000 Delivered-To: apmail-drill-commits-archive@drill.apache.org Received: (qmail 93849 invoked by uid 500); 2 Mar 2016 23:52:20 -0000 Mailing-List: contact commits-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: commits@drill.apache.org Delivered-To: mailing list commits@drill.apache.org Received: (qmail 93783 invoked by uid 99); 2 Mar 2016 23:52:20 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 02 Mar 2016 23:52:20 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9726FDFBD5; Wed, 2 Mar 2016 23:52:20 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: venki@apache.org To: commits@drill.apache.org Date: Wed, 02 Mar 2016 23:52:22 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [3/7] drill git commit: DRILL-4327: Fix rawtypes warnings in drill codebase http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java index 0cebd03..8702eb5 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java +++ b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java @@ -67,9 +67,6 @@ public class TestBuilder { // while this does work faster and use less memory, it can be harder to debug as all of the elements are not in a // single list private boolean highPerformanceComparison; - // for cases where the result set is just a single record, test writers can avoid creating a lot of small baseline - // files by providing a list of baseline values - private Object[] baselineValues; // column names for use with the baseline values protected String[] baselineColumns; // In cases where we need to verify larger datasets without the risk of running the baseline data through @@ -79,7 +76,7 @@ public class TestBuilder { // going with an approach of using this facility to validate the parts of the drill engine that could break in ways // that would affect the reading of baseline files (i.e. we need robust test for storage engines, project and casting that // use this interface) and then rely on the engine for the rest of the tests that will use the baseline queries. - private List baselineRecords; + private List> baselineRecords; private int expectedNumBatches = DrillTestWrapper.EXPECTED_BATCH_COUNT_NOT_SET; @@ -123,7 +120,7 @@ public class TestBuilder { throw new Exception("High performance comparison only available for ordered checks, to enforce this restriction, ordered() must be called first."); } return new DrillTestWrapper(this, allocator, query, queryType, baselineOptionSettingQueries, testOptionSettingQueries, - getValidationQueryType(), ordered, approximateEquality, highPerformanceComparison, baselineRecords, expectedNumBatches); + getValidationQueryType(), ordered, highPerformanceComparison, baselineRecords, expectedNumBatches); } public List> getExpectedSchema() { @@ -243,7 +240,6 @@ public class TestBuilder { public SchemaTestBuilder schemaBaseLine(List> expectedSchema) { assert expectedSchema != null : "The expected schema can be provided once"; assert baselineColumns == null : "The column information should be captured in expected schema, not baselineColumns"; - assert baselineValues == null && baselineRecords == null : "Since only schema will be compared in this test, no record is expected"; return new SchemaTestBuilder( allocator, @@ -270,7 +266,7 @@ public class TestBuilder { // indicate that the tests query should be checked for an empty result set public TestBuilder expectsEmptyResultSet() { unOrdered(); - baselineRecords = new ArrayList(); + baselineRecords = new ArrayList<>(); return this; } @@ -301,9 +297,9 @@ public class TestBuilder { throw new RuntimeException("Ordering not set, before specifying baseline data you must explicitly call the ordered() or unOrdered() method on the " + this.getClass().getSimpleName()); } if (baselineRecords == null) { - baselineRecords = new ArrayList(); + baselineRecords = new ArrayList<>(); } - Map ret = new HashMap(); + Map ret = new HashMap<>(); int i = 0; assertEquals("Must supply the same number of baseline values as columns.", baselineValues.length, baselineColumns.length); for (String s : baselineColumns) { @@ -328,7 +324,7 @@ public class TestBuilder { * @param materializedRecords - a list of maps representing materialized results * @return */ - public TestBuilder baselineRecords(List materializedRecords) { + public TestBuilder baselineRecords(List> materializedRecords) { this.baselineRecords = materializedRecords; return this; } @@ -439,6 +435,7 @@ public class TestBuilder { return this; } + @Override protected TestBuilder reset() { super.reset(); baselineTypeMap = null; @@ -447,6 +444,7 @@ public class TestBuilder { return this; } + @Override boolean typeInfoSet() { if (super.typeInfoSet() || baselineTypes != null) { return true; @@ -455,6 +453,7 @@ public class TestBuilder { } } + @Override String getValidationQuery() throws Exception { if (baselineColumns.length == 0) { throw new Exception("Baseline CSV files require passing column names, please call the baselineColumns() method on the test builder."); @@ -491,6 +490,7 @@ public class TestBuilder { return query; } + @Override protected UserBitShared.QueryType getValidationQueryType() throws Exception { return UserBitShared.QueryType.SQL; } @@ -505,19 +505,20 @@ public class TestBuilder { this.expectedSchema = expectedSchema; } + @Override public TestBuilder baselineColumns(String... columns) { assert false : "The column information should be captured in expected scheme, not baselineColumns"; return this; } @Override - public TestBuilder baselineRecords(List materializedRecords) { + public TestBuilder baselineRecords(List> materializedRecords) { assert false : "Since only schema will be compared in this test, no record is expected"; return this; } @Override - public TestBuilder baselineValues(Object[] objects) { + public TestBuilder baselineValues(Object... objects) { assert false : "Since only schema will be compared in this test, no record is expected"; return this; } @@ -548,10 +549,12 @@ public class TestBuilder { this.baselineColumns = new String[] {"*"}; } + @Override String getValidationQuery() { return "select " + Joiner.on(", ").join(baselineColumns) + " from cp.`" + baselineFilePath + "`"; } + @Override protected UserBitShared.QueryType getValidationQueryType() throws Exception { return UserBitShared.QueryType.SQL; } @@ -574,10 +577,12 @@ public class TestBuilder { this.baselineQueryType = baselineQueryType; } + @Override String getValidationQuery() { return baselineQuery; } + @Override protected UserBitShared.QueryType getValidationQueryType() throws Exception { return baselineQueryType; } @@ -585,6 +590,7 @@ public class TestBuilder { // This currently assumes that all explicit baseline queries will have fully qualified type information // if this changes, the baseline query can be run in a sub query with the implicit or explicit type passing // added on top of it, as is currently when done when reading a baseline file + @Override boolean typeInfoSet() { return true; } @@ -594,8 +600,8 @@ public class TestBuilder { /** * Convenience method to create a {@link JsonStringArrayList list} from the given values. */ - public static JsonStringArrayList listOf(Object... values) { - final JsonStringArrayList list = new JsonStringArrayList<>(); + public static JsonStringArrayList listOf(Object... values) { + final JsonStringArrayList list = new JsonStringArrayList<>(); for (Object value:values) { if (value instanceof CharSequence) { list.add(new Text(value.toString())); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java index 3771edd..09e4d9a 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java @@ -24,12 +24,11 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import java.math.BigDecimal; -import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import com.google.common.collect.Lists; import org.apache.commons.lang3.tuple.Pair; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos; @@ -38,6 +37,8 @@ import org.apache.drill.exec.planner.physical.PlannerSettings; import org.hamcrest.CoreMatchers; import org.junit.Test; +import com.google.common.collect.Lists; + // TODO - update framework to remove any dependency on the Drill engine for reading baseline result sets // currently using it with the assumption that the csv and json readers are well tested, and handling diverse // types in the test framework would require doing some redundant work to enable casting outside of Drill or @@ -79,7 +80,7 @@ public class TestFrameworkTest extends BaseTestQuery{ testBuilder() .sqlQuery(query) .schemaBaseLine(expectedSchema) - .baselineRecords(new ArrayList()) + .baselineRecords(Collections.>emptyList()) .build() .run(); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/HyperVectorValueIterator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/HyperVectorValueIterator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/HyperVectorValueIterator.java index 9ad72eb..156a965 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/HyperVectorValueIterator.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/HyperVectorValueIterator.java @@ -17,15 +17,15 @@ ******************************************************************************/ package org.apache.drill.exec; +import java.util.Iterator; + import org.apache.drill.exec.record.HyperVectorWrapper; import org.apache.drill.exec.record.MaterializedField; import org.apache.drill.exec.vector.ValueVector; -import java.util.Iterator; - public class HyperVectorValueIterator implements Iterator { private MaterializedField mf; - private HyperVectorWrapper hyperVector; + private HyperVectorWrapper hyperVector; private int indexInVectorList; private int indexInCurrentVector; private ValueVector currVec; @@ -34,7 +34,7 @@ public class HyperVectorValueIterator implements Iterator { // limit how many values will be read out of this iterator private long recordLimit; - public HyperVectorValueIterator(MaterializedField mf, HyperVectorWrapper hyperVector) { + public HyperVectorValueIterator(MaterializedField mf, HyperVectorWrapper hyperVector) { this.mf = mf; this.hyperVector = hyperVector; this.totalValues = 0; @@ -47,7 +47,7 @@ public class HyperVectorValueIterator implements Iterator { this.recordLimit = limit; } - public HyperVectorWrapper getHyperVector() { + public HyperVectorWrapper getHyperVector() { return hyperVector; } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlatten.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlatten.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlatten.java index a17aae0..fc42bb6 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlatten.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/flatten/TestFlatten.java @@ -21,7 +21,8 @@ import static org.apache.drill.TestBuilder.listOf; import static org.apache.drill.TestBuilder.mapOf; import static org.junit.Assert.assertEquals; -import com.google.common.collect.Lists; +import java.util.List; + import org.apache.drill.BaseTestQuery; import org.apache.drill.TestBuilder; import org.apache.drill.common.util.FileUtils; @@ -32,7 +33,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.util.List; +import com.google.common.collect.Lists; public class TestFlatten extends BaseTestQuery { @@ -118,7 +119,7 @@ public class TestFlatten extends BaseTestQuery { mapOf("nested_list_col", 999, "list_col", 9, "a", 1, "b",2) ); int i = 0; - for (JsonStringHashMap record : result) { + for (JsonStringHashMap record : result) { assertEquals(record, expectedResult.get(i)); i++; } @@ -136,9 +137,9 @@ public class TestFlatten extends BaseTestQuery { String flattenedDataColName) { List> output = Lists.newArrayList(); for (JsonStringHashMap incomingRecord : incomingRecords) { - List dataToFlatten = (List) incomingRecord.get(colToFlatten); + List dataToFlatten = (List) incomingRecord.get(colToFlatten); for (int i = 0; i < dataToFlatten.size(); i++) { - final JsonStringHashMap newRecord = new JsonStringHashMap(); + final JsonStringHashMap newRecord = new JsonStringHashMap<>(); newRecord.put(flattenedDataColName, dataToFlatten.get(i)); for (String s : incomingRecord.keySet()) { if (s.equals(colToFlatten)) { http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java index d56a65f..06b8d59 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java @@ -23,9 +23,6 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.List; -import mockit.Injectable; -import mockit.NonStrictExpectations; - import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.scanner.ClassPathScanner; import org.apache.drill.common.util.FileUtils; @@ -59,6 +56,9 @@ import com.google.common.base.Charsets; import com.google.common.collect.Lists; import com.google.common.io.Files; +import mockit.Injectable; +import mockit.NonStrictExpectations; + public class TestMergeJoin extends PopUnitTestBase { //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class); @@ -91,7 +91,7 @@ public class TestMergeJoin extends PopUnitTestBase { } System.out.println("\n"); for (int valueIdx = 0; valueIdx < exec.getRecordCount(); valueIdx++) { - final List row = new ArrayList(); + final List row = new ArrayList<>(); for (final ValueVector v : exec) { row.add(v.getAccessor().getObject(valueIdx)); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSort.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSort.java index 5066c83..6209b81 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSort.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/sort/TestSort.java @@ -29,8 +29,8 @@ import org.junit.Test; */ public class TestSort extends BaseTestQuery { - private static final JsonStringHashMap x = new JsonStringHashMap(); - private static final JsonStringArrayList repeated_map = new JsonStringArrayList<>(); + private static final JsonStringHashMap x = new JsonStringHashMap<>(); + private static final JsonStringArrayList> repeated_map = new JsonStringArrayList<>(); static { x.put("c", 1l); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java index b830f48..c3427f6 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java @@ -108,7 +108,7 @@ public class TestDirectoryExplorerUDFs extends PlanTestBase { excludedPatterns.toArray(excludedArray)); } - JsonStringArrayList list = new JsonStringArrayList(); + JsonStringArrayList list = new JsonStringArrayList<>(); list.add(new Text("1")); list.add(new Text("2")); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java index 38c6ebc..f6bcf97 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java @@ -20,7 +20,6 @@ package org.apache.drill.exec.record.vector; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import io.netty.buffer.DrillBuf; import java.nio.charset.Charset; @@ -67,6 +66,8 @@ import org.junit.Test; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import io.netty.buffer.DrillBuf; + public class TestValueVector extends ExecTest { //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestValueVector.class); @@ -744,7 +745,7 @@ the interface to load has changed final VectorVerifier noChild = new ChildVerifier(); final VectorVerifier offsetChild = new ChildVerifier(UInt4Holder.TYPE); - final ImmutableMap.Builder builder = ImmutableMap.builder(); + final ImmutableMap.Builder, VectorVerifier> builder = ImmutableMap.builder(); builder.put(UInt4Vector.class, noChild); builder.put(BitVector.class, noChild); builder.put(VarCharVector.class, offsetChild); @@ -752,14 +753,14 @@ the interface to load has changed builder.put(RepeatedListVector.class, new ChildVerifier(UInt4Holder.TYPE, Types.LATE_BIND_TYPE)); builder.put(MapVector.class, noChild); builder.put(RepeatedMapVector.class, offsetChild); - final ImmutableMap children = builder.build(); + final ImmutableMap, VectorVerifier> children = builder.build(); testVectors(new VectorVerifier() { @Override public void verify(ValueVector vector) throws Exception { - final Class klazz = vector.getClass(); + final Class klazz = vector.getClass(); final VectorVerifier verifier = children.get(klazz); verifier.verify(vector); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java index dadb850..cdc3162 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestAffinityCalculator.java @@ -79,7 +79,7 @@ public class TestAffinityCalculator extends ExecTest { } public LinkedList buildEndpoints(int numberOfEndpoints) { - LinkedList endPoints = new LinkedList(); + LinkedList endPoints = new LinkedList<>(); for (int i = 0; i < numberOfEndpoints; i++) { endPoints.add(CoordinationProtos.DrillbitEndpoint.newBuilder().setAddress("host" + i).build()); @@ -155,7 +155,7 @@ public class TestAffinityCalculator extends ExecTest { } ImmutableRangeMap map = blockMapBuilder.build(); long tB = System.nanoTime(); - System.out.println(String.format("Took %f ms to build range map", (float)(tB - tA) / 1e6)); + System.out.println(String.format("Took %f ms to build range map", (tB - tA) / 1e6)); } /* @Test http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java index 053b5cc..b6736d6 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java @@ -17,7 +17,15 @@ */ package org.apache.drill.exec.store.avro; -import com.google.common.base.Charsets; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.avro.Schema; import org.apache.avro.Schema.Type; import org.apache.avro.SchemaBuilder; @@ -27,14 +35,7 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import com.google.common.base.Charsets; /** * Utilities for generating Avro test data. @@ -45,9 +46,9 @@ public class AvroTestUtil { public static class AvroTestSetup { private String filePath; - private List expectedRecords; + private List> expectedRecords; - public AvroTestSetup(String filePath, List expectedRecords) { + public AvroTestSetup(String filePath, List> expectedRecords) { this.filePath = filePath; this.expectedRecords = expectedRecords; } @@ -59,15 +60,15 @@ public class AvroTestUtil { * by the Drill test builder to describe expected results. */ public static class AvroTestRecordWriter implements Closeable { - private final List expectedRecords; + private final List> expectedRecords; GenericData.Record currentAvroRecord; Map currentExpectedRecord; private Schema schema; - private final DataFileWriter writer; + private final DataFileWriter writer; private final String filePath; private AvroTestRecordWriter(Schema schema, File file) { - writer = new DataFileWriter(new GenericDatumWriter(schema)); + writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); } catch (IOException e) { @@ -112,7 +113,7 @@ public class AvroTestUtil { return filePath; } - public ListgetExpectedRecords() { + public List>getExpectedRecords() { return expectedRecords; } } @@ -184,7 +185,7 @@ public class AvroTestUtil { final File file = File.createTempFile("avro-primitive-test", ".avro"); file.deleteOnExit(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); @@ -230,7 +231,7 @@ public class AvroTestUtil { final File file = File.createTempFile("avro-primitive-test", ".avro"); file.deleteOnExit(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); @@ -273,7 +274,7 @@ public class AvroTestUtil { final Schema enumSchema = schema.getField("b_enum").schema(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); @@ -308,7 +309,7 @@ public class AvroTestUtil { .name("e_float_array").type().array().items().floatType().noDefault() .endRecord(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); @@ -316,24 +317,27 @@ public class AvroTestUtil { final GenericRecord record = new GenericData.Record(schema); record.put("a_string", "a_" + i); record.put("b_int", i); - - GenericArray array = new GenericData.Array(RECORD_COUNT, schema.getField("c_string_array").schema()); - for (int j = 0; j < RECORD_COUNT; j++) { - array.add(j, "c_string_array_" + i + "_" + j); + { + GenericArray array = new GenericData.Array<>(RECORD_COUNT, schema.getField("c_string_array").schema()); + for (int j = 0; j < RECORD_COUNT; j++) { + array.add(j, "c_string_array_" + i + "_" + j); + } + record.put("c_string_array", array); } - record.put("c_string_array", array); - - array = new GenericData.Array(RECORD_COUNT, schema.getField("d_int_array").schema()); - for (int j = 0; j < RECORD_COUNT; j++) { - array.add(j, i * j); + { + GenericArray array = new GenericData.Array<>(RECORD_COUNT, schema.getField("d_int_array").schema()); + for (int j = 0; j < RECORD_COUNT; j++) { + array.add(j, i * j); + } + record.put("d_int_array", array); } - record.put("d_int_array", array); - - array = new GenericData.Array(RECORD_COUNT, schema.getField("e_float_array").schema()); - for (int j = 0; j < RECORD_COUNT; j++) { - array.add(j, (float) (i * j)); + { + GenericArray array = new GenericData.Array<>(RECORD_COUNT, schema.getField("e_float_array").schema()); + for (int j = 0; j < RECORD_COUNT; j++) { + array.add(j, (float) (i * j)); + } + record.put("e_float_array", array); } - record.put("e_float_array", array); writer.append(record); } @@ -365,7 +369,7 @@ public class AvroTestUtil { final Schema nestedSchema = schema.getField("c_record").schema(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -409,7 +413,7 @@ public class AvroTestUtil { final Schema arraySchema = nestedSchema.getTypes().get(1); final Schema itemSchema = arraySchema.getElementType(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -419,7 +423,7 @@ public class AvroTestUtil { record.put("b_int", i); if (i % 2 == 0) { - GenericArray array = new GenericData.Array(1, arraySchema); + GenericArray array = new GenericData.Array<>(1, arraySchema); final GenericRecord nestedRecord = new GenericData.Record(itemSchema); nestedRecord.put("nested_1_string", "nested_1_string_" + i); nestedRecord.put("nested_1_int", i * i); @@ -448,7 +452,7 @@ public class AvroTestUtil { .name("c_map").type().optional().map().values(Schema.create(Type.STRING)) .endRecord(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -489,7 +493,7 @@ public class AvroTestUtil { final Schema arrayMapSchema = schema.getField("d_map").schema(); final Schema arrayItemSchema = arrayMapSchema.getTypes().get(1).getValueType(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -504,8 +508,8 @@ public class AvroTestUtil { c_map.put("key2", "nested_1_string_" + (i + 1 )); record.put("c_map", c_map); } else { - Map d_map = new HashMap<>(); - GenericArray array = new GenericData.Array(RECORD_COUNT, arrayItemSchema); + Map> d_map = new HashMap<>(); + GenericArray array = new GenericData.Array<>(RECORD_COUNT, arrayItemSchema); for (int j = 0; j < RECORD_COUNT; j++) { array.add((double)j); } @@ -543,7 +547,7 @@ public class AvroTestUtil { final Schema nestedSchema = schema.getField("c_record").schema(); final Schema optionalSchema = nestedSchema.getTypes().get(1); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -596,7 +600,7 @@ public class AvroTestUtil { final Schema nestedSchema = schema.getField("c_record").schema(); final Schema doubleNestedSchema = nestedSchema.getField("nested_1_record").schema(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); try { @@ -625,7 +629,6 @@ public class AvroTestUtil { return file.getAbsolutePath(); } - @SuppressWarnings({ "rawtypes", "unchecked" }) public static String generateLinkedList() throws Exception { final File file = File.createTempFile("avro-linkedlist", ".avro"); @@ -639,7 +642,7 @@ public class AvroTestUtil { .name("next").type().optional().type("LongList") .endRecord(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); writer.create(schema, file); GenericRecord previousRecord = null; try { @@ -673,7 +676,7 @@ public class AvroTestUtil { final File file = File.createTempFile("avro-primitive-test", ".avro"); file.deleteOnExit(); - final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); + final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema)); try { writer.create(schema, file); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java index 593e0db..b4a9e79 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java @@ -24,11 +24,9 @@ import java.util.HashMap; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.exec.store.ByteArrayUtil; -import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; - import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.bytes.DirectByteBufferAllocator; import org.apache.parquet.column.ColumnDescriptor; @@ -166,7 +164,7 @@ public class TestFileGenerator { CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); w.start(); - HashMap columnValuesWritten = new HashMap(); + HashMap columnValuesWritten = new HashMap<>(); int valsWritten; for (int k = 0; k < props.numberRowGroups; k++) { w.startBlock(props.recordsPerRowGroup); @@ -176,13 +174,13 @@ public class TestFileGenerator { for (FieldInfo fieldInfo : props.fields.values()) { if ( ! columnValuesWritten.containsKey(fieldInfo.name)) { - columnValuesWritten.put((String) fieldInfo.name, 0); + columnValuesWritten.put(fieldInfo.name, 0); valsWritten = 0; } else { valsWritten = columnValuesWritten.get(fieldInfo.name); } - String[] path1 = {(String) fieldInfo.name}; + String[] path1 = {fieldInfo.name}; ColumnDescriptor c1 = schema.getColumnDescription(path1); w.startColumn(c1, props.recordsPerRowGroup, codec); @@ -201,8 +199,8 @@ public class TestFileGenerator { new DirectByteBufferAllocator()); // for variable length binary fields int bytesNeededToEncodeLength = 4; - if ((int) fieldInfo.bitLength > 0) { - bytes = new byte[(int) Math.ceil(valsPerPage * (int) fieldInfo.bitLength / 8.0)]; + if (fieldInfo.bitLength > 0) { + bytes = new byte[(int) Math.ceil(valsPerPage * fieldInfo.bitLength / 8.0)]; } else { // the twelve at the end is to account for storing a 4 byte length with each value int totalValLength = ((byte[]) fieldInfo.values[0]).length + ((byte[]) fieldInfo.values[1]).length + ((byte[]) fieldInfo.values[2]).length + 3 * bytesNeededToEncodeLength; @@ -216,9 +214,9 @@ public class TestFileGenerator { } bytes = new byte[valsPerPage / 3 * totalValLength + leftOverBytes]; } - int bytesPerPage = (int) (valsPerPage * ((int) fieldInfo.bitLength / 8.0)); + int bytesPerPage = (int) (valsPerPage * (fieldInfo.bitLength / 8.0)); int bytesWritten = 0; - for (int z = 0; z < (int) fieldInfo.numberOfPages; z++, bytesWritten = 0) { + for (int z = 0; z < fieldInfo.numberOfPages; z++, bytesWritten = 0) { for (int i = 0; i < valsPerPage; i++) { repLevels.writeInteger(0); defLevels.writeInteger(1); @@ -244,7 +242,7 @@ public class TestFileGenerator { bytesWritten += ((byte[])fieldInfo.values[valsWritten % 3]).length + bytesNeededToEncodeLength; } else{ System.arraycopy( ByteArrayUtil.toByta(fieldInfo.values[valsWritten % 3]), - 0, bytes, i * ((int) fieldInfo.bitLength / 8), (int) fieldInfo.bitLength / 8); + 0, bytes, i * (fieldInfo.bitLength / 8), fieldInfo.bitLength / 8); } valsWritten++; } @@ -260,8 +258,8 @@ public class TestFileGenerator { currentBooleanByte = 0; } w.endColumn(); - columnValuesWritten.remove((String) fieldInfo.name); - columnValuesWritten.put((String) fieldInfo.name, valsWritten); + columnValuesWritten.remove(fieldInfo.name); + columnValuesWritten.put(fieldInfo.name, valsWritten); } w.endBlock(); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/testing/Controls.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/Controls.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/Controls.java index ef0e4a8..36ccee3 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/Controls.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/Controls.java @@ -17,10 +17,11 @@ */ package org.apache.drill.exec.testing; -import com.google.common.collect.Lists; +import java.util.List; + import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; -import java.util.List; +import com.google.common.collect.Lists; public class Controls { @@ -131,7 +132,7 @@ public class Controls { * @param nSkip number of times to skip before firing * @return this builder */ - public Builder addPause(final Class siteClass, final String desc, final int nSkip) { + public Builder addPause(final Class siteClass, final String desc, final int nSkip) { injections.add(ControlsInjectionUtil.createPause(siteClass, desc, nSkip)); return this; } @@ -144,7 +145,7 @@ public class Controls { * @param desc descriptor for the pause site in the site class * @return this builder */ - public Builder addPause(final Class siteClass, final String desc) { + public Builder addPause(final Class siteClass, final String desc) { return addPause(siteClass, desc, 0); } @@ -156,7 +157,7 @@ public class Controls { * @param nSkip number of times to skip before firing * @return this builder */ - public Builder addPauseOnBit(final Class siteClass, final String desc, + public Builder addPauseOnBit(final Class siteClass, final String desc, final DrillbitEndpoint endpoint, final int nSkip) { injections.add(ControlsInjectionUtil.createPauseOnBit(siteClass, desc, nSkip, endpoint)); return this; @@ -170,7 +171,7 @@ public class Controls { * @param desc descriptor for the pause site in the site class * @return this builder */ - public Builder addPauseOnBit(final Class siteClass, final String desc, + public Builder addPauseOnBit(final Class siteClass, final String desc, final DrillbitEndpoint endpoint) { return addPauseOnBit(siteClass, desc, endpoint, 0); } @@ -182,7 +183,7 @@ public class Controls { * @param desc descriptor for the latch in the site class * @return this builder */ - public Builder addLatch(final Class siteClass, final String desc) { + public Builder addLatch(final Class siteClass, final String desc) { injections.add(ControlsInjectionUtil.createLatch(siteClass, desc)); return this; } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java index d2aca37..3f6de15 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/ControlsInjectionUtil.java @@ -17,6 +17,12 @@ */ package org.apache.drill.exec.testing; +import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROLS_VALIDATOR; +import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROL_INJECTIONS; +import static org.junit.Assert.fail; + +import java.util.List; + import org.apache.drill.exec.client.DrillClient; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.UserBitShared; @@ -27,12 +33,6 @@ import org.apache.drill.exec.rpc.user.UserSession.QueryCountIncrementer; import org.apache.drill.exec.server.options.OptionManager; import org.apache.drill.exec.server.options.OptionValue; -import java.util.List; - -import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROLS_VALIDATOR; -import static org.apache.drill.exec.ExecConstants.DRILLBIT_CONTROL_INJECTIONS; -import static org.junit.Assert.fail; - /** * Static methods for constructing exception and pause injections for testing purposes. */ @@ -130,7 +130,7 @@ public class ControlsInjectionUtil { * Create a pause injection. Note this format is not directly accepted by the injection mechanism. Use the * {@link Controls} to build exceptions. */ - public static String createPause(final Class siteClass, final String desc, final int nSkip) { + public static String createPause(final Class siteClass, final String desc, final int nSkip) { return "{ \"type\" : \"pause\"," + "\"siteClass\" : \"" + siteClass.getName() + "\"," + "\"desc\" : \"" + desc + "\"," @@ -141,7 +141,7 @@ public class ControlsInjectionUtil { * Create a pause injection on a specific bit. Note this format is not directly accepted by the injection * mechanism. Use the {@link Controls} to build exceptions. */ - public static String createPauseOnBit(final Class siteClass, final String desc, final int nSkip, + public static String createPauseOnBit(final Class siteClass, final String desc, final int nSkip, final DrillbitEndpoint endpoint) { return "{ \"type\" : \"pause\"," + "\"siteClass\" : \"" + siteClass.getName() + "\"," @@ -155,7 +155,7 @@ public class ControlsInjectionUtil { * Create a latch injection. Note this format is not directly accepted by the injection mechanism. Use the * {@link Controls} to build exceptions. */ - public static String createLatch(final Class siteClass, final String desc) { + public static String createLatch(final Class siteClass, final String desc) { return "{ \"type\":\"latch\"," + "\"siteClass\":\"" + siteClass.getName() + "\"," + "\"desc\":\"" + desc + "\"}"; http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java ---------------------------------------------------------------------- diff --git a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java index b335c0f..4c7e6c1 100644 --- a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java +++ b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java @@ -108,7 +108,7 @@ public class ITTestShadedJar { try { Field f = ClassLoader.class.getDeclaredField("classes"); f.setAccessible(true); - Vector classes = (Vector) f.get(classLoader); + Vector> classes = (Vector>) f.get(classLoader); return classes.size(); } catch (Exception e) { System.out.println("Failure while loading class count."); @@ -120,7 +120,7 @@ public class ITTestShadedJar { try { Field f = ClassLoader.class.getDeclaredField("classes"); f.setAccessible(true); - Vector classes = (Vector) f.get(classLoader); + Vector> classes = (Vector>) f.get(classLoader); for (Class c : classes) { System.out.println(prefix + ": " + c.getName()); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---------------------------------------------------------------------- diff --git a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java index cf09be3..ed6e791 100644 --- a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java +++ b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java @@ -102,7 +102,7 @@ public abstract class BasicClient(connection)); } }); // http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---------------------------------------------------------------------- diff --git a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java index 8900034..27364af 100644 --- a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java +++ b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java @@ -92,7 +92,7 @@ public abstract class BasicServer(connection)); connect = true; // logger.debug("Server connection initialization completed."); @@ -104,7 +104,7 @@ public abstract class BasicServer extends ReadTimeoutHandler { + private class LogggingReadTimeoutHandler extends ReadTimeoutHandler { private final C connection; private final int timeoutSeconds; http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/codegen/templates/BasicTypeHelper.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java index dfc8e4a..a618cfd 100644 --- a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java +++ b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java @@ -73,7 +73,7 @@ public class BasicTypeHelper { } - public static Class getValueVectorClass(MinorType type, DataMode mode){ + public static Class getValueVectorClass(MinorType type, DataMode mode){ switch (type) { case UNION: return UnionVector.class; http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/codegen/templates/UnionVector.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/codegen/templates/UnionVector.java b/exec/vector/src/main/codegen/templates/UnionVector.java index dbdefba..f80bb25 100644 --- a/exec/vector/src/main/codegen/templates/UnionVector.java +++ b/exec/vector/src/main/codegen/templates/UnionVector.java @@ -236,7 +236,7 @@ public class UnionVector implements ValueVector { String name = v.getField().getType().getMinorType().name().toLowerCase(); MajorType type = v.getField().getType(); Preconditions.checkState(internalMap.getChild(name) == null, String.format("%s vector already exists", name)); - final ValueVector newVector = internalMap.addOrGet(name, type, (Class) BasicTypeHelper.getValueVectorClass(type.getMinorType(), type.getMode())); + final ValueVector newVector = internalMap.addOrGet(name, type, BasicTypeHelper.getValueVectorClass(type.getMinorType(), type.getMode())); v.makeTransferPair(newVector).transfer(); internalMap.putChild(name, newVector); addSubType(v.getField().getType().getMinorType()); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java index ea5e3ad..5fcecc6 100644 --- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java +++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java @@ -42,7 +42,7 @@ public class JsonStringArrayList extends ArrayList { if (!(obj instanceof List)) { return false; } - List other = (List) obj; + List other = (List) obj; return this.size() == other.size() && this.containsAll(other); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java index 22c927c..2ccb5ef 100644 --- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java +++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java @@ -46,7 +46,7 @@ public class JsonStringHashMap extends LinkedHashMap { if (!(obj instanceof Map)) { return false; } - Map other = (Map) obj; + Map other = (Map) obj; if (this.size() != other.size()) { return false; } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java index e79542b..3f40d4c 100644 --- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java +++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java @@ -107,7 +107,7 @@ public class ZeroVector implements ValueVector { } @Override - public Iterator iterator() { + public Iterator iterator() { return Iterators.emptyIterator(); } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java ---------------------------------------------------------------------- diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java index 13e9a9e..dbbd092 100644 --- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java +++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java @@ -84,16 +84,16 @@ public class PromotableWriter extends AbstractPromotableFieldWriter { state = State.SINGLE; vector = v; type = v.getField().getType().getMinorType(); - Class writerClass = BasicTypeHelper + Class writerClass = BasicTypeHelper .getWriterImpl(v.getField().getType().getMinorType(), v.getField().getDataMode()); if (writerClass.equals(SingleListWriter.class)) { writerClass = UnionListWriter.class; } - Class vectorClass = BasicTypeHelper.getValueVectorClass(v.getField().getType().getMinorType(), v.getField() + Class vectorClass = BasicTypeHelper.getValueVectorClass(v.getField().getType().getMinorType(), v.getField() .getDataMode()); try { - Constructor constructor = null; - for (Constructor c : writerClass.getConstructors()) { + Constructor constructor = null; + for (Constructor c : writerClass.getConstructors()) { if (c.getParameterTypes().length == 3) { constructor = c; } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/logical/src/main/java/org/apache/drill/common/logical/data/Limit.java ---------------------------------------------------------------------- diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/Limit.java b/logical/src/main/java/org/apache/drill/common/logical/data/Limit.java index 56ae8d9..3cc2603 100644 --- a/logical/src/main/java/org/apache/drill/common/logical/data/Limit.java +++ b/logical/src/main/java/org/apache/drill/common/logical/data/Limit.java @@ -55,7 +55,7 @@ public class Limit extends SingleInputOperator { } @Override - public NodeBuilder nodeBuilder() { + public NodeBuilder nodeBuilder() { return new LimitNodeBuilder(); //To change body of implemented methods use File | Settings | File Templates. } http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java ---------------------------------------------------------------------- diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java b/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java index 3343d4e..f1a4928 100644 --- a/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java +++ b/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperator.java @@ -48,7 +48,7 @@ public interface LogicalOperator extends GraphValue { public void registerAsSubscriber(LogicalOperator operator); - NodeBuilder nodeBuilder(); + NodeBuilder nodeBuilder(); public interface NodeBuilder { ObjectNode convert(ObjectMapper mapper, T operator, Integer inputId); http://git-wip-us.apache.org/repos/asf/drill/blob/2ffe3117/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java ---------------------------------------------------------------------- diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java b/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java index a213c8d..2350bb1 100644 --- a/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java +++ b/logical/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java @@ -49,7 +49,7 @@ public abstract class LogicalOperatorBase implements LogicalOperator{ } @Override - public NodeBuilder nodeBuilder() { + public NodeBuilder nodeBuilder() { // FIXME: Implement this on all logical operators throw new UnsupportedOperationException("Not yet implemented."); }