http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java index fc9775c..8e2ce96 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java @@ -65,7 +65,7 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { this.va = new VectorContainer(); } - public VectorAccessibleSerializable(WritableBatch batch, BufferAllocator allocator){ + public VectorAccessibleSerializable(WritableBatch batch, BufferAllocator allocator) { this(batch, null, allocator); } @@ -135,7 +135,6 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { writeToStream(output); } - /** * Serializes the VectorAccessible va and writes it to an output stream * @param output the OutputStream to write to @@ -153,29 +152,25 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { DrillBuf svBuf = null; Integer svCount = null; - if (svMode == BatchSchema.SelectionVectorMode.TWO_BYTE) - { + if (svMode == BatchSchema.SelectionVectorMode.TWO_BYTE) { svCount = sv2.getCount(); svBuf = sv2.getBuffer(); //this calls retain() internally } - try - { - /* Write the metadata to the file */ + try { + /* Write the metadata to the file */ batchDef.writeDelimitedTo(output); - /* If we have a selection vector, dump it to file first */ - if (svBuf != null) - { + /* If we have a selection vector, dump it to file first */ + if (svBuf != null) { svBuf.getBytes(0, output, svBuf.readableBytes()); sv2.setBuffer(svBuf); svBuf.release(); // sv2 now owns the buffer sv2.setRecordCount(svCount); } - /* Dump the array of ByteBuf's associated with the value vectors */ - for (DrillBuf buf : incomingBuffers) - { + /* Dump the array of ByteBuf's associated with the value vectors */ + for (DrillBuf buf : incomingBuffers) { /* dump the buffer into the OutputStream */ int bufLength = buf.readableBytes(); buf.getBytes(0, output, bufLength); @@ -184,8 +179,7 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { output.flush(); timerContext.stop(); - } catch (IOException e) - { + } catch (IOException e) { throw new RuntimeException(e); } finally { clear(); @@ -195,7 +189,9 @@ public class VectorAccessibleSerializable extends AbstractStreamSerializable { public void clear() { if (!retain) { batch.clear(); - if(sv2 != null) sv2.clear(); + if (sv2 != null) { + sv2.clear(); + } } }
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java index 7328257..99ead1c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java @@ -115,12 +115,11 @@ public class LocalCache implements DistributedCache { } private static BytesHolder serialize(Object obj, SerializationMode mode) { - if(obj instanceof String){ + if (obj instanceof String) { return new BytesHolder( ((String)obj).getBytes(Charsets.UTF_8)); } - try{ - switch(mode){ - + try{ + switch (mode) { case DRILL_SERIALIZIABLE: { ByteArrayDataOutput out = ByteStreams.newDataOutput(); OutputStream outputStream = DataOutputOutputStream.constructOutputStream(out); @@ -139,7 +138,7 @@ public class LocalCache implements DistributedCache { return new BytesHolder(( (Message) obj).toByteArray()); } - }catch(Exception e){ + } catch (Exception e) { throw new RuntimeException(e); } @@ -148,14 +147,13 @@ public class LocalCache implements DistributedCache { private static <V> V deserialize(BytesHolder b, SerializationMode mode, Class<V> clazz) { byte[] bytes = b.bytes; - try{ + try { if (clazz == String.class) { return (V) new String(bytes, Charsets.UTF_8); } switch (mode) { - case DRILL_SERIALIZIABLE: { InputStream inputStream = new ByteArrayInputStream(bytes); V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator); @@ -174,13 +172,15 @@ public class LocalCache implements DistributedCache { parser = (Parser<V>) f.get(null); } } - if (parser == null) throw new UnsupportedOperationException(String.format("Unable to find parser for class %s.", clazz.getName())); + if (parser == null) { + throw new UnsupportedOperationException(String.format("Unable to find parser for class %s.", clazz.getName())); + } InputStream inputStream = new ByteArrayInputStream(bytes); return parser.parseFrom(inputStream); } } - }catch(Exception e){ + } catch (Exception e) { throw new RuntimeException(e); } @@ -189,9 +189,10 @@ public class LocalCache implements DistributedCache { private static class BytesHolder { final byte[] bytes; - public BytesHolder(byte[] bytes){ + public BytesHolder(byte[] bytes) { this.bytes = bytes; } + @Override public int hashCode() { final int prime = 31; @@ -199,21 +200,25 @@ public class LocalCache implements DistributedCache { result = prime * result + Arrays.hashCode(bytes); return result; } + @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } BytesHolder other = (BytesHolder) obj; - if (!Arrays.equals(bytes, other.bytes)) + if (!Arrays.equals(bytes, other.bytes)) { return false; + } return true; } - } static class LocalDistributedMultiMapImpl<K, V> implements DistributedMultiMap<K, V> { @@ -288,13 +293,15 @@ public class LocalCache implements DistributedCache { @Override public V get(K key) { BytesHolder b = m.get(serialize(key, config.getMode())); - if(b == null) return null; + if (b == null) { + return null; + } return (V) deserialize(b, config.getMode(), config.getValueClass()); } @Override public Iterable<Entry<K, V>> getLocalEntries() { - return new Iterable<Entry<K, V>>(){ + return new Iterable<Entry<K, V>>() { @Override public Iterator<Entry<K, V>> iterator() { return new DeserializingTransformer(m.entrySet().iterator()); @@ -376,7 +383,6 @@ public class LocalCache implements DistributedCache { } - } public static class LocalCounterImpl implements Counter { @@ -397,4 +403,5 @@ public class LocalCache implements DistributedCache { return al.decrementAndGet(); } } + } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java index 510d637..74cc6a6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java @@ -88,11 +88,11 @@ public class DrillClient implements Closeable, ConnectionThrottle{ this(config, null); } - public DrillClient(DrillConfig config, ClusterCoordinator coordinator){ + public DrillClient(DrillConfig config, ClusterCoordinator coordinator) { this(config, coordinator, null); } - public DrillClient(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator){ + public DrillClient(DrillConfig config, ClusterCoordinator coordinator, BufferAllocator allocator) { this.ownsZkConnection = coordinator == null; this.ownsAllocator = allocator == null; this.allocator = allocator == null ? new TopLevelAllocator(config) : allocator; @@ -103,7 +103,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{ this.supportComplexTypes = config.getBoolean(ExecConstants.CLIENT_SUPPORT_COMPLEX_TYPES); } - public DrillConfig getConfig(){ + public DrillConfig getConfig() { return config; } @@ -139,7 +139,9 @@ public class DrillClient implements Closeable, ConnectionThrottle{ } public synchronized void connect(String connect, Properties props) throws RpcException { - if (connected) return; + if (connected) { + return; + } if (ownsZkConnection) { try { @@ -152,8 +154,9 @@ public class DrillClient implements Closeable, ConnectionThrottle{ if (props != null) { UserProperties.Builder upBuilder = UserProperties.newBuilder(); - for(String key : props.stringPropertyNames()) + for (String key : props.stringPropertyNames()) { upBuilder.addProperties(Property.newBuilder().setKey(key).setValue(props.getProperty(key))); + } this.props = upBuilder.build(); } @@ -210,10 +213,14 @@ public class DrillClient implements Closeable, ConnectionThrottle{ /** * Closes this client's connection to the server */ - public void close(){ - if(this.client != null) this.client.close(); - if(this.ownsAllocator && allocator != null) allocator.close(); - if(ownsZkConnection){ + public void close() { + if (this.client != null) { + this.client.close(); + } + if (this.ownsAllocator && allocator != null) { + allocator.close(); + } + if(ownsZkConnection) { try { this.clusterCoordinator.close(); } catch (IOException e) { @@ -240,7 +247,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{ return listener.getResults(); } - public DrillRpcFuture<Ack> cancelQuery(QueryId id){ + public DrillRpcFuture<Ack> cancelQuery(QueryId id) { logger.debug("Cancelling query {}", QueryIdHelper.getQueryId(id)); return client.send(RpcType.CANCEL_QUERY, id, Ack.class); } @@ -253,7 +260,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{ * @return a handle for the query result * @throws RpcException */ - public void runQuery(QueryType type, String plan, UserResultsListener resultsListener){ + public void runQuery(QueryType type, String plan, UserResultsListener resultsListener) { client.submitQuery(resultsListener, newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build()); } @@ -294,15 +301,15 @@ public class DrillClient implements Closeable, ConnectionThrottle{ public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) { // logger.debug("Result arrived. Is Last Chunk: {}. Full Result: {}", result.getHeader().getIsLastChunk(), result); results.add(result); - if(result.getHeader().getIsLastChunk()){ + if (result.getHeader().getIsLastChunk()) { future.set(results); } } public List<QueryResultBatch> getResults() throws RpcException{ - try{ + try { return future.get(); - }catch(Throwable t){ + } catch (Throwable t) { throw RpcException.mapException(t); } } @@ -328,7 +335,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{ getInner().setException(new RpcException(String.format("Failure connecting to server. Failure of type %s.", type.name()), t)); } - private SettableFuture<Void> getInner(){ + private SettableFuture<Void> getInner() { return (SettableFuture<Void>) delegate(); } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java index 54a5a3a..55d9cf3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java @@ -87,7 +87,7 @@ public class DumpCat { public void validate(String name, String value) throws ParameterException { try { int batch = Integer.parseInt(value); - if(batch < 0) { + if (batch < 0) { throw new ParameterException("Parameter " + name + " should be non-negative number."); } } catch (NumberFormatException e) { @@ -140,10 +140,11 @@ public class DumpCat { @Override public String toString() { String avgRecSizeStr = null; - if (this.rows>0) + if (this.rows>0) { avgRecSizeStr = String.format("Average Record Size : %d ", this.dataSize/this.rows); - else + } else { avgRecSizeStr = "Average Record Size : 0"; + } return String.format("Records : %d / %d \n", this.selectedRows, this.rows) + avgRecSizeStr + @@ -175,28 +176,29 @@ public class DumpCat { while (input.available() > 0) { VectorAccessibleSerializable vcSerializable = new VectorAccessibleSerializable(DumpCat.allocator); vcSerializable.readFromStream(input); - VectorContainer vectorContainer = (VectorContainer) vcSerializable.get(); + VectorContainer vectorContainer = (VectorContainer) vcSerializable.get(); - aggBatchMetaInfo.add(getBatchMetaInfo(vcSerializable)); + aggBatchMetaInfo.add(getBatchMetaInfo(vcSerializable)); - if (vectorContainer.getRecordCount() == 0) { - emptyBatchNum ++; - } + if (vectorContainer.getRecordCount() == 0) { + emptyBatchNum ++; + } - if (prevSchema != null && !vectorContainer.getSchema().equals(prevSchema)) - schemaChangeIdx.add(batchNum); + if (prevSchema != null && !vectorContainer.getSchema().equals(prevSchema)) { + schemaChangeIdx.add(batchNum); + } - prevSchema = vectorContainer.getSchema(); - batchNum ++; + prevSchema = vectorContainer.getSchema(); + batchNum ++; - vectorContainer.zeroVectors(); + vectorContainer.zeroVectors(); } - /* output the summary stat */ - System.out.println(String.format("Total # of batches: %d", batchNum)); - //output: rows, selectedRows, avg rec size, total data size. - System.out.println(aggBatchMetaInfo.toString()); - System.out.println(String.format("Empty batch : %d", emptyBatchNum)); + /* output the summary stat */ + System.out.println(String.format("Total # of batches: %d", batchNum)); + //output: rows, selectedRows, avg rec size, total data size. + System.out.println(aggBatchMetaInfo.toString()); + System.out.println(String.format("Empty batch : %d", emptyBatchNum)); System.out.println(String.format("Schema changes : %d", schemaChangeIdx.size())); System.out.println(String.format("Schema change batch index : %s", schemaChangeIdx.toString())); } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java index 3302e7c..1ed3cb3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java @@ -94,7 +94,9 @@ public class PrintingResultsListener implements UserResultsListener { public int await() throws Exception { latch.await(); - if(exception != null) throw exception; + if (exception != null) { + throw exception; + } return count.get(); } @@ -106,4 +108,5 @@ public class PrintingResultsListener implements UserResultsListener { public void queryIdArrived(QueryId queryId) { this.queryId = queryId; } -} \ No newline at end of file + +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java index 2f5c027..3a6dad0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java @@ -106,7 +106,7 @@ public class QuerySubmitter { RemoteServiceSet serviceSet = null; Drillbit[] drillbits = null; - try{ + try { if (local) { serviceSet = RemoteServiceSet.getLocalServiceSet(); drillbits = new Drillbit[bits]; @@ -133,10 +133,12 @@ public class QuerySubmitter { } catch(Throwable th) { System.err.println("Query Failed due to : " + th.getMessage()); return -1; - }finally{ - if(client != null) client.close(); - if(local){ - for(Drillbit b : drillbits){ + } finally { + if (client != null) { + client.close(); + } + if (local) { + for (Drillbit b : drillbits) { b.close(); } serviceSet.close(); @@ -151,7 +153,7 @@ public class QuerySubmitter { String[] queries; QueryType queryType; type = type.toLowerCase(); - switch(type) { + switch (type) { case "sql": queryType = QueryType.SQL; queries = plan.trim().split(";"); @@ -171,7 +173,7 @@ public class QuerySubmitter { Format outputFormat; format = format.toLowerCase(); - switch(format) { + switch (format) { case "csv": outputFormat = Format.CSV; break; @@ -201,4 +203,5 @@ public class QuerySubmitter { return 0; } + } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java index 98c51c6..f4a3cc9 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/AbstractClassCompiler.java @@ -33,14 +33,16 @@ public abstract class AbstractClassCompiler { public byte[][] getClassByteCode(ClassNames className, String sourceCode) throws CompileException, IOException, ClassNotFoundException, ClassTransformationException { - if(getLogger().isDebugEnabled()){ + if (getLogger().isDebugEnabled()) { getLogger().debug("Compiling (source size={}):\n{}", DrillStringUtils.readable(sourceCode.length()), prefixLineNumbers(sourceCode)); } return getByteCode(className, sourceCode); } protected String prefixLineNumbers(String code) { - if (!debug) return code; + if (!debug) { + return code; + } StringBuilder out = new StringBuilder(); int i = 1; @@ -49,7 +51,7 @@ public abstract class AbstractClassCompiler { out.append(i++); int numLength = out.length() - start; out.append(":"); - for (int spaces = 0; spaces < 7 - numLength; ++spaces){ + for (int spaces = 0; spaces < 7 - numLength; ++spaces) { out.append(" "); } out.append(line); http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ByteCodeLoader.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ByteCodeLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ByteCodeLoader.java index 704a199..c11d02d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ByteCodeLoader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ByteCodeLoader.java @@ -40,8 +40,9 @@ class ByteCodeLoader { @Override public byte[] load(String path) throws ClassTransformationException, IOException { URL u = this.getClass().getResource(path); - if (u == null) + if (u == null) { throw new ClassTransformationException(String.format("Unable to find TemplateClass at path %s", path)); + } return Resources.toByteArray(u); } }; @@ -51,11 +52,14 @@ class ByteCodeLoader { return byteCode.get(path); } catch (ExecutionException e) { Throwable c = e.getCause(); - if (c instanceof ClassTransformationException) + if (c instanceof ClassTransformationException) { throw (ClassTransformationException) c; - if (c instanceof IOException) + } + if (c instanceof IOException) { throw (IOException) c; + } throw new ClassTransformationException(c); } } + } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java index bb24b57..2d69ca3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java @@ -61,11 +61,11 @@ public class ClassTransformer { String.format("The new name of a class cannot start with the old name of a class, otherwise class renaming will cause problems. Precompiled class name %s. Generated class name %s", precompiled, generated)); } - public ClassSet getChild(String precompiled, String generated){ + public ClassSet getChild(String precompiled, String generated) { return new ClassSet(this, precompiled, generated); } - public ClassSet getChild(String precompiled){ + public ClassSet getChild(String precompiled) { return new ClassSet(this, precompiled, precompiled.replace(this.precompiled.dot, this.generated.dot)); } @@ -81,41 +81,49 @@ public class ClassTransformer { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } ClassSet other = (ClassSet) obj; if (generated == null) { - if (other.generated != null) + if (other.generated != null) { return false; - } else if (!generated.equals(other.generated)) + } + } else if (!generated.equals(other.generated)) { return false; + } if (parent == null) { - if (other.parent != null) + if (other.parent != null) { return false; - } else if (!parent.equals(other.parent)) + } + } else if (!parent.equals(other.parent)) { return false; + } if (precompiled == null) { - if (other.precompiled != null) + if (other.precompiled != null) { return false; - } else if (!precompiled.equals(other.precompiled)) + } + } else if (!precompiled.equals(other.precompiled)) { return false; + } return true; } - } - public static class ClassNames{ + public static class ClassNames { public final String dot; public final String slash; public final String clazz; - public ClassNames(String className){ + public ClassNames(String className) { dot = className; slash = className.replace('.', FileUtils.separatorChar); clazz = FileUtils.separatorChar + slash + ".class"; @@ -133,28 +141,37 @@ public class ClassTransformer { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } ClassNames other = (ClassNames) obj; if (clazz == null) { - if (other.clazz != null) + if (other.clazz != null) { return false; - } else if (!clazz.equals(other.clazz)) + } + } else if (!clazz.equals(other.clazz)) { return false; + } if (dot == null) { - if (other.dot != null) + if (other.dot != null) { return false; - } else if (!dot.equals(other.dot)) + } + } else if (!dot.equals(other.dot)) { return false; + } if (slash == null) { - if (other.slash != null) + if (other.slash != null) { return false; - } else if (!slash.equals(other.slash)) + } + } else if (!slash.equals(other.slash)) { return false; + } return true; } } @@ -179,7 +196,7 @@ public class ClassTransformer { long totalBytecodeSize = 0; Map<String, ClassNode> classesToMerge = Maps.newHashMap(); - for(byte[] clazz : implementationClasses) { + for (byte[] clazz : implementationClasses) { totalBytecodeSize += clazz.length; ClassNode node = getClassNodeFromByteCode(clazz); classesToMerge.put(node.name, node); @@ -191,14 +208,16 @@ public class ClassTransformer { while ( !names.isEmpty() ) { final ClassSet nextSet = names.removeFirst(); - if (namesCompleted.contains(nextSet)) continue; + if (namesCompleted.contains(nextSet)) { + continue; + } final ClassNames nextPrecompiled = nextSet.precompiled; final byte[] precompiledBytes = byteCodeLoader.getClassByteCodeFromPath(nextPrecompiled.clazz); ClassNames nextGenerated = nextSet.generated; ClassNode generatedNode = classesToMerge.get(nextGenerated.slash); MergedClassResult result = MergeAdapter.getMergedClass(nextSet, precompiledBytes, generatedNode); - for(String s : result.innerClasses) { + for (String s : result.innerClasses) { s = s.replace(FileUtils.separatorChar, '.'); names.add(nextSet.getChild(s)); } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java index 024fd01..acc32b5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/DrillJavaFileObject.java @@ -83,8 +83,9 @@ final class DrillJavaFileObject extends SimpleJavaFileObject { @Override public CharSequence getCharContent(final boolean ignoreEncodingErrors) throws IOException { - if (sourceCode == null) + if (sourceCode == null) { throw new UnsupportedOperationException("This instance of DrillJavaFileObject is not an input object."); + } return sourceCode; } @@ -106,4 +107,4 @@ final class DrillJavaFileObject extends SimpleJavaFileObject { } } -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java index f056489..2e101dc 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/JDKClassCompiler.java @@ -74,8 +74,12 @@ class JDKClassCompiler extends AbstractClassCompiler { Throwable cause = rte.getCause(); if (cause != null) { cause = cause.getCause(); - if (cause instanceof CompileException) throw (CompileException) cause; - if (cause instanceof IOException) throw (IOException) cause; + if (cause instanceof CompileException) { + throw (CompileException) cause; + } + if (cause instanceof IOException) { + throw (IOException) cause; + } } throw rte; } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java index 945c94a..6a6be50 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java @@ -64,11 +64,15 @@ class MergeAdapter extends ClassVisitor { super(Opcodes.ASM4, cv); this.classToMerge = cn; this.set = set; - for(Object o : classToMerge.methods){ + for (Object o : classToMerge.methods) { String name = ((MethodNode)o).name; - if(name.equals("<init>")) continue; - if(name.equals(SignatureHolder.DRILL_INIT_METHOD)) hasInit = true; - mergingNames.add( name); + if (name.equals("<init>")) { + continue; + } + if (name.equals(SignatureHolder.DRILL_INIT_METHOD)) { + hasInit = true; + } + mergingNames.add(name); } } @@ -99,9 +103,9 @@ class MergeAdapter extends ClassVisitor { public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) { // use the access and names of the impl class. this.name = name; - if(name.contains("$")){ + if (name.contains("$")) { super.visit(version, access, name, signature, superName, interfaces); - }else{ + } else { super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL, name, signature, superName, interfaces); } @@ -111,7 +115,6 @@ class MergeAdapter extends ClassVisitor { @Override public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions) { - // finalize all methods. // skip all abstract methods as they should have implementations. @@ -120,17 +123,17 @@ class MergeAdapter extends ClassVisitor { // logger.debug("Skipping copy of '{}()' since it is abstract or listed elsewhere.", arg1); return null; } - if(signature != null){ + if (signature != null) { signature = signature.replace(set.precompiled.slash, set.generated.slash); } - // if( (access & Modifier.PUBLIC) == 0){ + // if ((access & Modifier.PUBLIC) == 0) { // access = access ^ Modifier.PUBLIC ^ Modifier.PROTECTED | Modifier.PRIVATE; // } MethodVisitor mv = super.visitMethod(access, name, desc, signature, exceptions); if (!name.equals("<init>")) { access = access | Modifier.FINAL; - }else{ - if(hasInit){ + } else { + if (hasInit) { return new DrillInitMethodVisitor(this.name, mv); } } @@ -150,7 +153,9 @@ class MergeAdapter extends ClassVisitor { for (Iterator<?> it = classToMerge.methods.iterator(); it.hasNext();) { MethodNode mn = (MethodNode) it.next(); - if (mn.name.equals("<init>")) continue; + if (mn.name.equals("<init>")) { + continue; + } String[] exceptions = new String[mn.exceptions.size()]; mn.exceptions.toArray(exceptions); @@ -161,7 +166,9 @@ class MergeAdapter extends ClassVisitor { // mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv, new // SimpleRemapper("org.apache.drill.exec.compile.ExampleTemplate", "Bunky"))); ClassSet top = set; - while(top.parent != null) top = top.parent; + while (top.parent != null) { + top = top.parent; + } mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv, new SimpleRemapper(top.precompiled.slash, top.generated.slash))); } @@ -173,7 +180,6 @@ class MergeAdapter extends ClassVisitor { return super.visitField(access, name, desc, signature, value); } - public static class MergedClassResult{ public byte[] bytes; public Collection<String> innerClasses; @@ -183,7 +189,6 @@ class MergeAdapter extends ClassVisitor { this.innerClasses = innerClasses; } - } public static MergedClassResult getMergedClass(ClassSet set, byte[] precompiledClass, ClassNode generatedClass) throws IOException{ @@ -193,15 +198,15 @@ class MergeAdapter extends ClassVisitor { ClassWriter writer = new ClassWriter(ClassWriter.COMPUTE_FRAMES); RemapClasses re = new RemapClasses(set); - try{ -// if(generatedClass != null){ + try { +// if(generatedClass != null) { // ClassNode generatedMerged = new ClassNode(); // generatedClass.accept(new ValueHolderReplacementVisitor(generatedMerged)); // generatedClass = generatedMerged; // } ClassVisitor remappingAdapter = new RemappingClassAdapter(writer, re); ClassVisitor visitor = remappingAdapter; - if(generatedClass != null){ + if (generatedClass != null) { visitor = new MergeAdapter(set, remappingAdapter, generatedClass); } ClassReader tReader = new ClassReader(precompiledClass); @@ -212,7 +217,7 @@ class MergeAdapter extends ClassVisitor { // Files.write(outputClass, new File(String.format("/src/scratch/drill-generated-classes/%s-output.class", set.generated.dot))); return new MergedClassResult(outputClass, re.getInnerClasses()); - }catch(Error | RuntimeException e){ + } catch (Error | RuntimeException e) { logger.error("Failure while merging classes.", e); throw e; } @@ -228,7 +233,9 @@ class MergeAdapter extends ClassVisitor { super(); this.current = set; ClassSet top = set; - while(top.parent != null) top = top.parent; + while (top.parent != null) { + top = top.parent; + } this.top = top; } @@ -239,7 +246,7 @@ class MergeAdapter extends ClassVisitor { if (typeName.startsWith(top.precompiled.slash)) { // write down all the sub classes. - if (typeName.startsWith(current.precompiled.slash + "$")){ + if (typeName.startsWith(current.precompiled.slash + "$")) { innerClasses.add(typeName); } @@ -258,31 +265,31 @@ class MergeAdapter extends ClassVisitor { Exception e = null; String error = ""; - try{ - ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES); - ClassVisitor cv = new CheckClassAdapter(cw, true); - node.accept(cv); + try { + ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES); + ClassVisitor cv = new CheckClassAdapter(cw, true); + node.accept(cv); - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - CheckClassAdapter.verify(new ClassReader(cw.toByteArray()), false, pw); + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + CheckClassAdapter.verify(new ClassReader(cw.toByteArray()), false, pw); - error = sw.toString(); - }catch(Exception ex){ + error = sw.toString(); + } catch (Exception ex) { e = ex; } - if(!error.isEmpty() || e != null){ + if (!error.isEmpty() || e != null) { StringWriter sw2 = new StringWriter(); PrintWriter pw2 = new PrintWriter(sw2); TraceClassVisitor v = new TraceClassVisitor(pw2); node.accept(v); - if(e != null){ + if (e != null) { throw new RuntimeException("Failure validating class. ByteCode: \n" + sw2.toString() + "\n\n====ERRROR====\n" + error, e); - }else{ + } else { throw new RuntimeException("Failure validating class. ByteCode: \n" + sw2.toString() + "\n\n====ERRROR====\n" + error); } - } } -} \ No newline at end of file + +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java index 398d109..e1ac7a8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java @@ -76,19 +76,21 @@ public class QueryClassLoader extends URLClassLoader { compilerSelector = new ClassCompilerSelector(config, sessionOptions); } - public long getNextClassIndex(){ + public long getNextClassIndex() { return index.getAndIncrement(); } public void injectByteCode(String className, byte[] classBytes) throws IOException { - if(customClasses.containsKey(className)) throw new IOException(String.format("The class defined {} has already been loaded.", className)); + if (customClasses.containsKey(className)) { + throw new IOException(String.format("The class defined {} has already been loaded.", className)); + } customClasses.put(className, classBytes); } @Override protected Class<?> findClass(String className) throws ClassNotFoundException { byte[] ba = customClasses.get(className); - if(ba != null){ + if (ba != null) { return this.defineClass(className, ba, 0, ba.length); }else{ return super.findClass(className); http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/InstructionModifier.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/InstructionModifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/InstructionModifier.java index e736aab..4585bd8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/InstructionModifier.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/InstructionModifier.java @@ -61,8 +61,9 @@ public class InstructionModifier extends MethodVisitor { private ReplacingBasicValue popCurrent(boolean includeReturnVals) { // for vararg, we could try to pop an empty stack. TODO: handle this better. - if (list.currentFrame.getStackSize() == 0) + if (list.currentFrame.getStackSize() == 0) { return null; + } Object o = list.currentFrame.pop(); if (o instanceof ReplacingBasicValue) { @@ -76,8 +77,9 @@ public class InstructionModifier extends MethodVisitor { private ReplacingBasicValue getReturn() { Object o = list.nextFrame.getStack(list.nextFrame.getStackSize() - 1); - if (o instanceof ReplacingBasicValue) + if (o instanceof ReplacingBasicValue) { return (ReplacingBasicValue) o; + } return null; } @@ -85,8 +87,9 @@ public class InstructionModifier extends MethodVisitor { public void visitInsn(int opcode) { switch (opcode) { case Opcodes.DUP: - if (popCurrent() != null) + if (popCurrent() != null) { return; + } } super.visitInsn(opcode); } @@ -111,14 +114,14 @@ public class InstructionModifier extends MethodVisitor { @Override public void visitVarInsn(int opcode, int var) { ReplacingBasicValue v; - if(opcode == Opcodes.ASTORE && (v = popCurrent(true)) != null){ - if(!v.isFunctionReturn){ + if (opcode == Opcodes.ASTORE && (v = popCurrent(true)) != null) { + if (!v.isFunctionReturn) { ValueHolderSub from = oldToNew.get(v.getIndex()); ReplacingBasicValue current = local(var); // if local var is set, then transfer to it to the existing holders in the local position. - if(current != null){ - if(oldToNew.get(current.getIndex()).iden() == from.iden()){ + if (current != null) { + if (oldToNew.get(current.getIndex()).iden() == from.iden()) { int targetFirst = oldToNew.get(current.index).first(); from.transfer(this, targetFirst); return; @@ -126,9 +129,9 @@ public class InstructionModifier extends MethodVisitor { } // if local var is not set, then check map to see if existing holders are mapped to local var. - if(oldLocalToFirst.containsKey(var)){ + if (oldLocalToFirst.containsKey(var)) { ValueHolderSub sub = oldToNew.get(oldLocalToFirst.lget()); - if(sub.iden() == from.iden()){ + if (sub.iden() == from.iden()) { // if they are, then transfer to that. from.transfer(this, oldToNew.get(oldLocalToFirst.lget()).first()); return; @@ -139,13 +142,13 @@ public class InstructionModifier extends MethodVisitor { // map from variables to global space for future use. oldLocalToFirst.put(var, v.getIndex()); - }else{ + } else { // this is storage of a function return, we need to map the fields to the holder spots. int first; - if(oldLocalToFirst.containsKey(var)){ + if (oldLocalToFirst.containsKey(var)) { first = oldToNew.get(oldLocalToFirst.lget()).first(); v.iden.transferToLocal(adder, first); - }else{ + } else { first = v.iden.createLocalAndTrasfer(adder); } ValueHolderSub from = v.iden.getHolderSubWithDefinedLocals(first); @@ -153,14 +156,12 @@ public class InstructionModifier extends MethodVisitor { v.disableFunctionReturn(); } - }else if(opcode == Opcodes.ALOAD && (v = getReturn()) != null){ - + } else if (opcode == Opcodes.ALOAD && (v = getReturn()) != null) { // noop. - }else{ + } else { super.visitVarInsn(opcode, var); } - } void directVarInsn(int opcode, int var) { @@ -176,10 +177,10 @@ public class InstructionModifier extends MethodVisitor { // pop twice for put. v = popCurrent(true); if (v != null) { - if(v.isFunctionReturn){ + if (v.isFunctionReturn) { super.visitFieldInsn(opcode, owner, name, desc); return; - }else{ + } else { // we are trying to store a replaced variable in an external context, we need to generate an instance and // transfer it out. ValueHolderSub sub = oldToNew.get(v.getIndex()); @@ -197,8 +198,6 @@ public class InstructionModifier extends MethodVisitor { sub.addInsn(name, this, opcode); return; } - - } super.visitFieldInsn(opcode, owner, name, desc); @@ -246,14 +245,14 @@ public class InstructionModifier extends MethodVisitor { } private void checkArg(String name, ReplacingBasicValue obj) { - if (obj == null) + if (obj == null) { return; + } throw new IllegalStateException( String .format( "Holder types are not allowed to be passed between methods. Ran across problem attempting to invoke method '%s' on line number %d", name, lastLineNumber)); - } } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderIden.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderIden.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderIden.java index 1e10eeb..a0ce390 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderIden.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderIden.java @@ -38,8 +38,8 @@ class ValueHolderIden { Field[] fields = c.getFields(); List<Field> fldList = Lists.newArrayList(); - for(Field f : fields){ - if(!Modifier.isStatic(f.getModifiers())) { + for (Field f : fields) { + if (!Modifier.isStatic(f.getModifiers())) { fldList.add(f); } } @@ -48,7 +48,7 @@ class ValueHolderIden { this.names = new String[fldList.size()]; fieldMap = new ObjectIntOpenHashMap<String>(); int i =0; - for(Field f : fldList){ + for (Field f : fldList) { types[i] = Type.getType(f.getType()); names[i] = f.getName(); fieldMap.put(f.getName(), i); @@ -56,8 +56,8 @@ class ValueHolderIden { } } - private static void initType(int index, Type t, DirectSorter v){ - switch(t.getSort()){ + private static void initType(int index, Type t, DirectSorter v) { + switch(t.getSort()) { case Type.BOOLEAN: case Type.BYTE: case Type.CHAR: @@ -97,30 +97,28 @@ class ValueHolderIden { } return new ValueHolderSub(first); - } - public ValueHolderSub getHolderSubWithDefinedLocals(int first){ + public ValueHolderSub getHolderSubWithDefinedLocals(int first) { return new ValueHolderSub(first); } - private int dup(Type t){ + private int dup(Type t) { return t.getSize() == 1 ? Opcodes.DUP : Opcodes.DUP2; } - public void transferToLocal(DirectSorter adder, int localVariable){ + public void transferToLocal(DirectSorter adder, int localVariable) { for (int i = 0; i < types.length; i++) { Type t = types[i]; - if(i + 1 < types.length) adder.visitInsn(dup(t)); // don't dup for last value. + if (i + 1 < types.length) { + adder.visitInsn(dup(t)); // don't dup for last value. + } adder.visitFieldInsn(Opcodes.GETFIELD, type.getInternalName(), names[i], t.getDescriptor()); adder.directVarInsn(t.getOpcode(Opcodes.ISTORE), localVariable+i); } } - - - - public int createLocalAndTrasfer(DirectSorter adder){ + public int createLocalAndTrasfer(DirectSorter adder) { int first = 0; for (int i = 0; i < types.length; i++) { Type t = types[i]; @@ -141,29 +139,31 @@ class ValueHolderIden { this.first = first; } - public ValueHolderIden iden(){ + public ValueHolderIden iden() { return ValueHolderIden.this; } - public void init(DirectSorter mv){ + public void init(DirectSorter mv) { for (int i = 0; i < types.length; i++) { initType(first+i, types[i], mv); } } - public int size(){ + public int size() { return types.length; } - public int first(){ + public int first() { return first; } - public void updateFirst(int newFirst){ + public void updateFirst(int newFirst) { this.first = newFirst; } private int field(String name, InstructionModifier mv) { - if (!fieldMap.containsKey(name)) throw new IllegalArgumentException(String.format("Unknown name '%s' on line %d.", name, mv.lastLineNumber)); + if (!fieldMap.containsKey(name)) { + throw new IllegalArgumentException(String.format("Unknown name '%s' on line %d.", name, mv.lastLineNumber)); + } return fieldMap.lget(); } @@ -178,9 +178,11 @@ class ValueHolderIden { } } - public void transfer(InstructionModifier mv, int newStart){ - if(first == newStart) return; - for(int i =0; i < types.length; i++){ + public void transfer(InstructionModifier mv, int newStart) { + if (first == newStart) { + return; + } + for (int i =0; i < types.length; i++) { mv.directVarInsn(types[i].getOpcode(Opcodes.ILOAD), first + i); mv.directVarInsn(types[i].getOpcode(Opcodes.ISTORE), newStart + i); } @@ -193,7 +195,7 @@ class ValueHolderIden { mv.directVarInsn(t.getOpcode(analogOpcode), first + f); } - public void transferToExternal(DirectSorter adder, String owner, String name, String desc){ + public void transferToExternal(DirectSorter adder, String owner, String name, String desc) { // create a new object and assign it to the desired field. adder.visitTypeInsn(Opcodes.NEW, type.getInternalName()); @@ -212,10 +214,8 @@ class ValueHolderIden { // lastly we save it to the desired field. adder.visitFieldInsn(Opcodes.PUTFIELD, owner, name, desc); - } } - -} \ No newline at end of file +} http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java index a7b5680..9df346c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java @@ -26,7 +26,7 @@ import com.google.common.collect.Iterators; import com.thoughtworks.paranamer.AnnotationParanamer; import com.thoughtworks.paranamer.Paranamer; -public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{ +public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument> { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeGeneratorMethod.class); private final String methodName; @@ -35,7 +35,7 @@ public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{ private final Class<?>[] exs; private final Method underlyingMethod; - public CodeGeneratorMethod(String name, Class<?> returnType){ + public CodeGeneratorMethod(String name, Class<?> returnType) { this.methodName = name; this.returnType = returnType; this.exs = new Class<?>[0]; @@ -43,18 +43,22 @@ public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{ this.arguments = new CodeGeneratorArgument[0]; } - public CodeGeneratorMethod(Method m){ + public CodeGeneratorMethod(Method m) { this.underlyingMethod = m; this.methodName = m.getName(); this.returnType = m.getReturnType(); // Paranamer para = new BytecodeReadingParanamer(); Paranamer para = new AnnotationParanamer(); String[] parameterNames = para.lookupParameterNames(m, true); - if(parameterNames == null) throw new RuntimeException(String.format("Unable to read the parameter names for method %s. This is likely due to the class files not including the appropriate debugging information. Look up java -g for more information.", m)); + if (parameterNames == null) { + throw new RuntimeException(String.format("Unable to read the parameter names for method %s. This is likely due to the class files not including the appropriate debugging information. Look up java -g for more information.", m)); + } Class<?>[] types = m.getParameterTypes(); - if(parameterNames.length != types.length) throw new RuntimeException(String.format("Unexpected number of parameter names %s. Expected %s on method %s.", Arrays.toString(parameterNames), Arrays.toString(types), m.toGenericString())); + if (parameterNames.length != types.length) { + throw new RuntimeException(String.format("Unexpected number of parameter names %s. Expected %s on method %s.", Arrays.toString(parameterNames), Arrays.toString(types), m.toGenericString())); + } arguments = new CodeGeneratorArgument[parameterNames.length]; - for(int i =0 ; i < parameterNames.length; i++){ + for (int i = 0 ; i < parameterNames.length; i++) { arguments[i] = new CodeGeneratorArgument(parameterNames[i], types[i]); } exs = m.getExceptionTypes(); @@ -67,7 +71,7 @@ public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{ return returnType; } - public Iterable<Class<?>> getThrowsIterable(){ + public Iterable<Class<?>> getThrowsIterable() { return ImmutableList.copyOf(exs); } @@ -81,5 +85,4 @@ public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{ return "CodeGeneratorMethod [" + underlyingMethod.toGenericString() + "]"; } - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java index 80aaae8..fbc586f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java @@ -42,7 +42,7 @@ public class MappingSet { this("inIndex", "outIndex", new GeneratorMapping[] { mapping, mapping }); } - public boolean hasEmbeddedConstant(){ + public boolean hasEmbeddedConstant() { return constant == current; } @@ -98,16 +98,18 @@ public class MappingSet { public void enterChild() { assert current == mappings[mappingIndex]; mappingIndex++; - if (mappingIndex >= mappings.length) + if (mappingIndex >= mappings.length) { throw new IllegalStateException("This generator does not support mappings beyond"); + } current = mappings[mappingIndex]; } public void exitChild() { assert current == mappings[mappingIndex]; mappingIndex--; - if (mappingIndex < 0) + if (mappingIndex < 0) { throw new IllegalStateException("You tried to traverse higher than the provided mapping provides."); + } current = mappings[mappingIndex]; } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java index d397acc..7fe8e3b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java @@ -29,7 +29,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ +public class SignatureHolder implements Iterable<CodeGeneratorMethod> { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SignatureHolder.class); private final Class<?> signature; @@ -40,17 +40,19 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ public static final String DRILL_INIT_METHOD = "__DRILL_INIT__"; public static final CodeGeneratorMethod DRILL_INIT = new CodeGeneratorMethod(DRILL_INIT_METHOD, void.class); - public static SignatureHolder getHolder(Class<?> signature){ + public static SignatureHolder getHolder(Class<?> signature) { List<SignatureHolder> innerClasses = Lists.newArrayList(); - for(Class<?> inner : signature.getClasses()){ + for (Class<?> inner : signature.getClasses()) { SignatureHolder h = getHolder(inner); - if(h.childHolders.length > 0 || h.methods.length > 0) innerClasses.add(h); + if (h.childHolders.length > 0 || h.methods.length > 0) { + innerClasses.add(h); + } } return new SignatureHolder(signature, innerClasses.toArray(new SignatureHolder[innerClasses.size()])); } - private SignatureHolder(Class<?> signature, SignatureHolder[] childHolders){ + private SignatureHolder(Class<?> signature, SignatureHolder[] childHolders) { this.childHolders = childHolders; this.signature = signature; Map<String, Integer> newMap = Maps.newHashMap(); @@ -58,30 +60,33 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ List<CodeGeneratorMethod> methodHolders = Lists.newArrayList(); Method[] reflectMethods = signature.getDeclaredMethods(); - for(Method m : reflectMethods){ - if( (m.getModifiers() & Modifier.ABSTRACT) == 0 && m.getAnnotation(RuntimeOverridden.class) == null) continue; + for (Method m : reflectMethods) { + if ( (m.getModifiers() & Modifier.ABSTRACT) == 0 && m.getAnnotation(RuntimeOverridden.class) == null) { + continue; + } methodHolders.add(new CodeGeneratorMethod(m)); } methods = new CodeGeneratorMethod[methodHolders.size()+1]; - for(int i =0; i < methodHolders.size(); i++){ + for (int i =0; i < methodHolders.size(); i++) { methods[i] = methodHolders.get(i); Integer old = newMap.put(methods[i].getMethodName(), i); - if(old != null) throw new IllegalStateException(String.format("Attempting to add a method with name %s when there is already one method of that name in this class that is set to be runtime generated.", methods[i].getMethodName())); + if (old != null) { + throw new IllegalStateException(String.format("Attempting to add a method with name %s when there is already one method of that name in this class that is set to be runtime generated.", methods[i].getMethodName())); + } } methods[methodHolders.size()] = DRILL_INIT; newMap.put(DRILL_INIT.getMethodName(), methodHolders.size()); methodMap = ImmutableMap.copyOf(newMap); - } - public Class<?> getSignatureClass(){ + public Class<?> getSignatureClass() { return signature; } - public CodeGeneratorMethod get(int i){ + public CodeGeneratorMethod get(int i) { return methods[i]; } @@ -90,7 +95,7 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ return Iterators.forArray(methods); } - public int size(){ + public int size() { return methods.length; } @@ -99,10 +104,9 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ return childHolders; } - - public int get(String method){ + public int get(String method) { Integer meth = methodMap.get(method); - if(meth == null){ + if (meth == null) { throw new IllegalStateException(String.format("Unknown method requested of name %s.", method)); } return meth; @@ -115,5 +119,4 @@ public class SignatureHolder implements Iterable<CodeGeneratorMethod>{ + (methods != null ? Arrays.asList(methods).subList(0, Math.min(methods.length, maxLen)) : null) + "]"; } - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java index 2ddf280..035c1aa 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java @@ -31,7 +31,7 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import com.google.common.collect.Maps; -public class LocalClusterCoordinator extends ClusterCoordinator{ +public class LocalClusterCoordinator extends ClusterCoordinator { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class); private volatile Map<RegistrationHandle, DrillbitEndpoint> endpoints = Maps.newConcurrentMap(); @@ -57,7 +57,9 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ @Override public void unregister(RegistrationHandle handle) { - if(handle == null) return; + if(handle == null) { + return; + } endpoints.remove(handle); } @@ -67,7 +69,6 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ return endpoints.values(); } - private class Handle implements RegistrationHandle{ UUID id = UUID.randomUUID(); @@ -82,14 +83,26 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ @Override public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } Handle other = (Handle) obj; - if (!getOuterType().equals(other.getOuterType())) return false; + if (!getOuterType().equals(other.getOuterType())) { + return false; + } if (id == null) { - if (other.id != null) return false; - } else if (!id.equals(other.id)) return false; + if (other.id != null) { + return false; + } + } else if (!id.equals(other.id)) { + return false; + } return true; } @@ -99,7 +112,6 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ } - @Override public DistributedSemaphore getSemaphore(String name, int maximumLeases) { semaphores.putIfAbsent(name, new LocalSemaphore(maximumLeases)); @@ -111,20 +123,19 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ private final Semaphore inner; private final LocalLease lease = new LocalLease(); - public LocalSemaphore(int size){ + public LocalSemaphore(int size) { inner = new Semaphore(size); } @Override public DistributedLease acquire(long timeout, TimeUnit unit) throws Exception { - if(!inner.tryAcquire(timeout, unit)){ + if(!inner.tryAcquire(timeout, unit)) { return null; }else{ return lease; } } - private class LocalLease implements DistributedLease{ @Override @@ -135,5 +146,4 @@ public class LocalClusterCoordinator extends ClusterCoordinator{ } } - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java index 76ad90b..7f538d2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java @@ -75,7 +75,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator { // check if this is a complex zk string. If so, parse into components. Matcher m = ZK_COMPLEX_STRING.matcher(connect); - if(m.matches()){ + if(m.matches()) { connect = m.group(1); zkRoot = m.group(2); clusterId = m.group(3); @@ -101,7 +101,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator { .build(); } - public CuratorFramework getCurator(){ + public CuratorFramework getCurator() { return curator; } @@ -112,9 +112,11 @@ public class ZKClusterCoordinator extends ClusterCoordinator { serviceCache.start(); serviceCache.addListener(new ZKListener()); - if(millisToWait != 0){ + if(millisToWait != 0) { boolean success = this.initialConnection.await(millisToWait, TimeUnit.MILLISECONDS); - if(!success) throw new IOException(String.format("Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.", millisToWait)); + if (!success) { + throw new IOException(String.format("Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.", millisToWait)); + } }else{ this.initialConnection.await(); } @@ -126,7 +128,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator { @Override public void stateChanged(CuratorFramework client, ConnectionState newState) { - if(newState == ConnectionState.CONNECTED){ + if(newState == ConnectionState.CONNECTED) { ZKClusterCoordinator.this.initialConnection.countDown(); client.getConnectionStateListenable().removeListener(this); } @@ -166,7 +168,9 @@ public class ZKClusterCoordinator extends ClusterCoordinator { @Override public void unregister(RegistrationHandle handle) { - if (!(handle instanceof ZKRegistrationHandle)) throw new UnsupportedOperationException("Unknown handle type: " + handle.getClass().getName()); + if (!(handle instanceof ZKRegistrationHandle)) { + throw new UnsupportedOperationException("Unknown handle type: " + handle.getClass().getName()); + } ZKRegistrationHandle h = (ZKRegistrationHandle) handle; try { @@ -223,4 +227,5 @@ public class ZKClusterCoordinator extends ClusterCoordinator { .serializer(DrillServiceInstanceHelper.SERIALIZER) .build(); } + } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java index a1132d7..8b6e1e3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java @@ -139,8 +139,9 @@ public class View { @JsonIgnore public boolean hasStar() { for (FieldType field : fields) { - if (StarColumnHelper.isNonPrefixedStarColumn(field.name)) + if (StarColumnHelper.isNonPrefixedStarColumn(field.name)) { return true; + } } return false; } @@ -165,6 +166,4 @@ public class View { return workspaceSchemaPath; } - - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java index 69d68bb..e0c4296 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java @@ -81,7 +81,7 @@ public class ClassGenerator<T>{ private int labelIndex = 0; private MappingSet mappings; - public static MappingSet getDefaultMapping(){ + public static MappingSet getDefaultMapping() { return new MappingSet("inIndex", "outIndex", DEFAULT_CONSTANT_MAP, DEFAULT_SCALAR_MAP); } @@ -94,29 +94,29 @@ public class ClassGenerator<T>{ this.evaluationVisitor = eval; this.model = model; blocks = (LinkedList<JBlock>[]) new LinkedList[sig.size()]; - for(int i =0; i < sig.size(); i++){ + for (int i =0; i < sig.size(); i++) { blocks[i] = Lists.newLinkedList(); } rotateBlock(); - for(SignatureHolder child : signature.getChildHolders()){ + for (SignatureHolder child : signature.getChildHolders()) { String innerClassName = child.getSignatureClass().getSimpleName(); JDefinedClass innerClazz = clazz._class(Modifier.FINAL + Modifier.PRIVATE, innerClassName); innerClasses.put(innerClassName, new ClassGenerator<>(codeGenerator, mappingSet, child, eval, innerClazz, model)); } } - public ClassGenerator<T> getInnerGenerator(String name){ + public ClassGenerator<T> getInnerGenerator(String name) { ClassGenerator<T> inner = innerClasses.get(name); Preconditions.checkNotNull(inner); return inner; } - public MappingSet getMappingSet(){ + public MappingSet getMappingSet() { return mappings; } - public void setMappingSet(MappingSet mappings){ + public void setMappingSet(MappingSet mappings) { this.mappings = mappings; } @@ -124,30 +124,30 @@ public class ClassGenerator<T>{ return codeGenerator; } - private GeneratorMapping getCurrentMapping(){ + private GeneratorMapping getCurrentMapping() { return mappings.getCurrentMapping(); } - public JBlock getBlock(String methodName){ + public JBlock getBlock(String methodName) { JBlock blk = this.blocks[sig.get(methodName)].getLast(); Preconditions.checkNotNull(blk, "Requested method name of %s was not available for signature %s.", methodName, this.sig); return blk; } - public JBlock getBlock(BlockType type){ + public JBlock getBlock(BlockType type) { return getBlock(getCurrentMapping().getMethodName(type)); } - public JBlock getSetupBlock(){ + public JBlock getSetupBlock() { return getBlock(getCurrentMapping().getMethodName(BlockType.SETUP)); } - public JBlock getEvalBlock(){ + public JBlock getEvalBlock() { return getBlock(getCurrentMapping().getMethodName(BlockType.EVAL)); } - public JBlock getResetBlock(){ + public JBlock getResetBlock() { return getBlock(getCurrentMapping().getMethodName(BlockType.RESET)); } - public JBlock getCleanupBlock(){ + public JBlock getCleanupBlock() { return getBlock(getCurrentMapping().getMethodName(BlockType.CLEANUP)); } @@ -165,11 +165,11 @@ public class ClassGenerator<T>{ return getEvalBlock().label(prefix + labelIndex ++); } - public JVar declareVectorValueSetupAndMember(String batchName, TypedFieldId fieldId){ + public JVar declareVectorValueSetupAndMember(String batchName, TypedFieldId fieldId) { return declareVectorValueSetupAndMember( DirectExpression.direct(batchName), fieldId); } - public JVar declareVectorValueSetupAndMember(DirectExpression batchName, TypedFieldId fieldId){ + public JVar declareVectorValueSetupAndMember(DirectExpression batchName, TypedFieldId fieldId) { final ValueVectorSetup setup = new ValueVectorSetup(batchName, fieldId); // JVar var = this.vvDeclaration.get(setup); // if(var != null) return var; @@ -178,7 +178,7 @@ public class ClassGenerator<T>{ JClass vvClass = model.ref(valueVectorClass); JClass retClass = vvClass; String vectorAccess = "getValueVector"; - if(fieldId.isHyperReader()){ + if (fieldId.isHyperReader()) { retClass = retClass.array(); vectorAccess = "getValueVectors"; } @@ -191,7 +191,7 @@ public class ClassGenerator<T>{ JVar fieldArr = b.decl(model.INT.array(), "fieldIds" + index++, JExpr.newArray(model.INT, fieldId.getFieldIds().length)); int[] fieldIndices = fieldId.getFieldIds(); - for(int i = 0; i < fieldIndices.length; i++){ + for (int i = 0; i < fieldIndices.length; i++) { b.assign(fieldArr.component(JExpr.lit(i)), JExpr.lit(fieldIndices[i])); } @@ -213,18 +213,20 @@ public class ClassGenerator<T>{ return vv; } - public HoldingContainer addExpr(LogicalExpression ex){ + public HoldingContainer addExpr(LogicalExpression ex) { return addExpr(ex, true); } - public HoldingContainer addExpr(LogicalExpression ex, boolean rotate){ + public HoldingContainer addExpr(LogicalExpression ex, boolean rotate) { // logger.debug("Adding next write {}", ex); - if(rotate) rotateBlock(); + if (rotate) { + rotateBlock(); + } return evaluationVisitor.addExpr(ex, this); } - public void rotateBlock(){ - for(LinkedList<JBlock> b : blocks){ + public void rotateBlock() { + for (LinkedList<JBlock> b : blocks) { b.add(new JBlock(true, true)); } } @@ -249,11 +251,11 @@ public class ClassGenerator<T>{ if (blocksInMethod > MAX_BLOCKS_IN_FUNCTION) { JMethod inner = clazz.method(JMod.PRIVATE, model._ref(method.getReturnType()), method.getMethodName() + methodIndex); JInvocation methodCall = JExpr.invoke(inner); - for(CodeGeneratorArgument arg : method){ + for (CodeGeneratorArgument arg : method) { inner.param(arg.getType(), arg.getName()); methodCall.arg(JExpr.direct(arg.getName())); } - for(Class<?> c : method.getThrowsIterable()){ + for (Class<?> c : method.getThrowsIterable()) { inner._throws(model.ref(c)); } inner._throws(SchemaChangeException.class); @@ -286,32 +288,32 @@ public class ClassGenerator<T>{ return "v" + index++; } - public String getNextVar(String prefix){ + public String getNextVar(String prefix) { return prefix + index++; } - public JVar declareClassField(String prefix, JType t){ + public JVar declareClassField(String prefix, JType t) { return clazz.field(JMod.NONE, t, prefix + index++); } - public JVar declareClassField(String prefix, JType t, JExpression init){ + public JVar declareClassField(String prefix, JType t, JExpression init) { return clazz.field(JMod.NONE, t, prefix + index++, init); } - public HoldingContainer declare(MajorType t){ + public HoldingContainer declare(MajorType t) { return declare(t, true); } - public HoldingContainer declare(MajorType t, boolean includeNewInstance){ + public HoldingContainer declare(MajorType t, boolean includeNewInstance) { JType holderType = getHolderType(t); JVar var; - if(includeNewInstance){ + if (includeNewInstance) { var = getEvalBlock().decl(holderType, "out" + index, JExpr._new(holderType)); - }else{ + } else { var = getEvalBlock().decl(holderType, "out" + index); } JFieldRef outputSet = null; - if(t.getMode() == DataMode.OPTIONAL){ + if (t.getMode() == DataMode.OPTIONAL) { outputSet = var.ref("isSet"); } index++; @@ -347,23 +349,30 @@ public class ClassGenerator<T>{ @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } ValueVectorSetup other = (ValueVectorSetup) obj; if (batch == null) { - if (other.batch != null) + if (other.batch != null) { return false; - } else if (!batch.equals(other.batch)) + } + } else if (!batch.equals(other.batch)) { return false; + } if (fieldId == null) { - if (other.fieldId != null) + if (other.fieldId != null) { return false; - } else if (!fieldId.equals(other.fieldId)) + } + } else if (!fieldId.equals(other.fieldId)) { return false; + } return true; } @@ -396,7 +405,7 @@ public class ClassGenerator<T>{ return this.isReader; } - public boolean isSingularRepeated(){ + public boolean isSingularRepeated() { return singularRepeated; } @@ -405,7 +414,7 @@ public class ClassGenerator<T>{ return this; } - public JFieldRef f(String name){ + public JFieldRef f(String name) { return holder.ref(name); } @@ -421,7 +430,7 @@ public class ClassGenerator<T>{ return value; } - public MajorType getMajorType(){ + public MajorType getMajorType() { return type; } @@ -430,11 +439,11 @@ public class ClassGenerator<T>{ return isSet; } - public boolean isOptional(){ + public boolean isOptional() { return type.getMode() == DataMode.OPTIONAL; } - public boolean isRepeated(){ + public boolean isRepeated() { return type.getMode() == DataMode.REPEATED; } @@ -443,7 +452,7 @@ public class ClassGenerator<T>{ } } - public JType getHolderType(MajorType t){ + public JType getHolderType(MajorType t) { return TypeHelper.getHolderType(model, t.getMinorType(), t.getMode()); } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java index 73c5980..4495ffe 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java @@ -93,7 +93,7 @@ public class CodeGenerator<T> { return generatedCode; } - public String getGeneratedCode(){ + public String getGeneratedCode() { return generatedCode; } @@ -111,12 +111,12 @@ public class CodeGenerator<T> { } public static <T> ClassGenerator<T> getRoot(TemplateClassDefinition<T> definition, - FunctionImplementationRegistry funcRegistry){ + FunctionImplementationRegistry funcRegistry) { return get(definition, funcRegistry).getRoot(); } public static <T> ClassGenerator<T> getRoot(MappingSet mappingSet, TemplateClassDefinition<T> definition, - FunctionImplementationRegistry funcRegistry){ + FunctionImplementationRegistry funcRegistry) { return get(mappingSet, definition, funcRegistry).getRoot(); } @@ -136,27 +136,31 @@ public class CodeGenerator<T> { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } CodeGenerator other = (CodeGenerator) obj; if (definition == null) { - if (other.definition != null) + if (other.definition != null) { return false; - } else if (!definition.equals(other.definition)) + } + } else if (!definition.equals(other.definition)) { return false; + } if (generatedCode == null) { - if (other.generatedCode != null) + if (other.generatedCode != null) { return false; - } else if (!generatedCode.equals(other.generatedCode)) + } + } else if (!generatedCode.equals(other.generatedCode)) { return false; + } return true; } - - - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java index bbf00c7..c4c3e7a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DirectExpression.java @@ -48,20 +48,24 @@ public class DirectExpression extends JExpressionImpl{ @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } DirectExpression other = (DirectExpression) obj; if (source == null) { - if (other.source != null) + if (other.source != null) { return false; - } else if (!source.equals(other.source)) + } + } else if (!source.equals(other.source)) { return false; + } return true; } - } http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/676f5df6/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java index 1e67158..a5b7bee 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java @@ -83,9 +83,9 @@ public class EvaluationVisitor { public HoldingContainer addExpr(LogicalExpression e, ClassGenerator<?> generator) { Set<LogicalExpression> constantBoundaries; - if(generator.getMappingSet().hasEmbeddedConstant()){ + if (generator.getMappingSet().hasEmbeddedConstant()) { constantBoundaries = Collections.emptySet(); - }else{ + } else { constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(e); } return e.accept(new ConstantFilter(constantBoundaries), generator); @@ -104,7 +104,7 @@ public class EvaluationVisitor { ClassGenerator<?> generator) throws RuntimeException { if (op.getName().equals("booleanAnd")) { return visitBooleanAnd(op, generator); - }else if(op.getName().equals("booleanOr")) { + } else if(op.getName().equals("booleanOr")) { return visitBooleanOr(op, generator); } else { throw new UnsupportedOperationException("BooleanOperator can only be booleanAnd, booleanOr. You are using " + op.getName()); @@ -119,8 +119,9 @@ public class EvaluationVisitor { JVar[] workspaceVars = holder.renderStart(generator, null); - if (holder.isNested()) + if (holder.isNested()) { generator.getMappingSet().enterChild(); + } HoldingContainer[] args = new HoldingContainer[holderExpr.args.size()]; for (int i = 0; i < holderExpr.args.size(); i++) { @@ -129,8 +130,9 @@ public class EvaluationVisitor { holder.renderMiddle(generator, args, workspaceVars); - if (holder.isNested()) + if (holder.isNested()) { generator.getMappingSet().exitChild(); + } return holder.renderEnd(generator, args, workspaceVars); } @@ -392,8 +394,9 @@ public class EvaluationVisitor { if (seg.isArray()) { // stop once we get to the last segment and the final type is neither complex nor repeated (map, list, repeated list). // In case of non-complex and non-repeated type, we return Holder, in stead of FieldReader. - if (seg.isLastPath() && !complex && !repeated) + if (seg.isLastPath() && !complex && !repeated) { break; + } JVar list = generator.declareClassField("list", generator.getModel()._ref(FieldReader.class)); eval.assign(list, expr); @@ -466,11 +469,13 @@ public class EvaluationVisitor { */ private boolean isNullReaderLikely(PathSegment seg, boolean complexOrRepeated) { while (seg != null) { - if (seg.isArray() && !seg.isLastPath()) + if (seg.isArray() && !seg.isLastPath()) { return true; + } - if (seg.isLastPath() && complexOrRepeated) + if (seg.isLastPath() && complexOrRepeated) { return true; + } seg = seg.getChild(); } @@ -624,9 +629,9 @@ public class EvaluationVisitor { JBlock earlyExit = null; if (arg.isOptional()) { earlyExit = eval._if(arg.getIsSet().eq(JExpr.lit(1)).cand(arg.getValue().ne(JExpr.lit(1))))._then(); - if(e == null){ + if (e == null) { e = arg.getIsSet(); - }else{ + } else { e = e.mul(arg.getIsSet()); } } else { @@ -687,9 +692,9 @@ public class EvaluationVisitor { JBlock earlyExit = null; if (arg.isOptional()) { earlyExit = eval._if(arg.getIsSet().eq(JExpr.lit(1)).cand(arg.getValue().eq(JExpr.lit(1))))._then(); - if(e == null){ + if (e == null) { e = arg.getIsSet(); - }else{ + } else { e = e.mul(arg.getIsSet()); } } else { @@ -1033,4 +1038,5 @@ public class EvaluationVisitor { .setConstant(true); } } + }