Skip to content

Commit 982be6b

Browse files
committed
fix styling
1 parent 68d8a4f commit 982be6b

File tree

4 files changed

+114
-110
lines changed

4 files changed

+114
-110
lines changed

exec/java-exec/src/main/java/org/apache/drill/exec/cache/CustomCacheManager.java

Lines changed: 63 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -13,67 +13,66 @@
1313
import com.github.benmanes.caffeine.cache.Caffeine;
1414

1515
public class CustomCacheManager {
16-
private static final Logger logger = LoggerFactory.getLogger(CustomCacheManager.class);
17-
18-
private static Cache<String, PhysicalPlan> queryCache;
19-
private static Cache<CacheKey, RelNode> transformCache;
20-
21-
private static int queryMaxEntries;
22-
private static int queryTtlMinutes;
23-
private static int transformMaxEntries;
24-
private static int transformTtlMinutes;
25-
26-
static {
27-
loadConfig();
28-
}
29-
30-
private static void loadConfig() {
31-
DrillConfig config = DrillConfig.create();
32-
33-
queryMaxEntries = getConfigInt(config, "custom.cache.query.max_entries", 100);
34-
queryTtlMinutes = getConfigInt(config, "custom.cache.query.ttl_minutes", 300);
35-
transformMaxEntries = getConfigInt(config, "custom.cache.transform.max_entries", 100);
36-
transformTtlMinutes = getConfigInt(config, "custom.cache.transform.ttl_minutes", 300);
37-
38-
queryCache = Caffeine.newBuilder()
39-
.maximumSize(queryMaxEntries)
40-
.expireAfterWrite(queryTtlMinutes, TimeUnit.MINUTES)
41-
.recordStats()
42-
.build();
43-
44-
transformCache = Caffeine.newBuilder()
45-
.maximumSize(transformMaxEntries)
46-
.expireAfterWrite(transformTtlMinutes, TimeUnit.MINUTES)
47-
.recordStats()
48-
.build();
49-
}
50-
51-
private static int getConfigInt(DrillConfig config, String path, int defaultValue) {
52-
return config.hasPath(path) ? config.getInt(path) : defaultValue;
53-
}
54-
55-
public static PhysicalPlan getQueryPlan(String sql) {
56-
return queryCache.getIfPresent(sql);
57-
}
58-
59-
public static void putQueryPlan(String sql, PhysicalPlan plan) {
60-
queryCache.put(sql, plan);
61-
}
62-
63-
public static RelNode getTransformedPlan(CacheKey key) {
64-
return transformCache.getIfPresent(key);
65-
}
66-
67-
public static void putTransformedPlan(CacheKey key, RelNode plan) {
68-
transformCache.put(key, plan);
69-
}
70-
71-
public static void logCacheStats() {
72-
logger.info("Query Cache Stats: " + queryCache.stats());
73-
logger.info("Query Cache Size: " + queryCache.estimatedSize());
74-
75-
logger.info("Transform Cache Stats: " + transformCache.stats());
76-
logger.info("Transform Cache Size: " + transformCache.estimatedSize());
77-
}
78-
79-
}
16+
private static final Logger logger = LoggerFactory.getLogger(CustomCacheManager.class);
17+
18+
private static Cache<String, PhysicalPlan> queryCache;
19+
private static Cache<CacheKey, RelNode> transformCache;
20+
21+
private static int queryMaxEntries;
22+
private static int queryTtlMinutes;
23+
private static int transformMaxEntries;
24+
private static int transformTtlMinutes;
25+
26+
static {
27+
loadConfig();
28+
}
29+
30+
private static void loadConfig() {
31+
DrillConfig config = DrillConfig.create();
32+
33+
queryMaxEntries = getConfigInt(config, "custom.cache.query.max_entries", 100);
34+
queryTtlMinutes = getConfigInt(config, "custom.cache.query.ttl_minutes", 300);
35+
transformMaxEntries = getConfigInt(config, "custom.cache.transform.max_entries", 100);
36+
transformTtlMinutes = getConfigInt(config, "custom.cache.transform.ttl_minutes", 300);
37+
38+
queryCache = Caffeine.newBuilder()
39+
.maximumSize(queryMaxEntries)
40+
.expireAfterWrite(queryTtlMinutes, TimeUnit.MINUTES)
41+
.recordStats()
42+
.build();
43+
44+
transformCache = Caffeine.newBuilder()
45+
.maximumSize(transformMaxEntries)
46+
.expireAfterWrite(transformTtlMinutes, TimeUnit.MINUTES)
47+
.recordStats()
48+
.build();
49+
}
50+
51+
private static int getConfigInt(DrillConfig config, String path, int defaultValue) {
52+
return config.hasPath(path) ? config.getInt(path) : defaultValue;
53+
}
54+
55+
public static PhysicalPlan getQueryPlan(String sql) {
56+
return queryCache.getIfPresent(sql);
57+
}
58+
59+
public static void putQueryPlan(String sql, PhysicalPlan plan) {
60+
queryCache.put(sql, plan);
61+
}
62+
63+
public static RelNode getTransformedPlan(CacheKey key) {
64+
return transformCache.getIfPresent(key);
65+
}
66+
67+
public static void putTransformedPlan(CacheKey key, RelNode plan) {
68+
transformCache.put(key, plan);
69+
}
70+
71+
public static void logCacheStats() {
72+
logger.info("Query Cache Stats: " + queryCache.stats());
73+
logger.info("Query Cache Size: " + queryCache.estimatedSize());
74+
75+
logger.info("Transform Cache Stats: " + transformCache.stats());
76+
logger.info("Transform Cache Size: " + transformCache.estimatedSize());
77+
}
78+
}

exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java

Lines changed: 19 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -182,7 +182,7 @@ private static PhysicalPlan convertPlan(QueryContext context, String sql, Pointe
182182
private static PhysicalPlan getPhysicalPlan(QueryContext context, String sql, Pointer<String> textPlan,
183183
long retryAttempts) throws ForemanSetupException, RelConversionException, IOException, ValidationException {
184184
try {
185-
logger.info("DrillSqlWorker.getPhysicalPlan() is called {}", retryAttempts);
185+
logger.info("DrillSqlWorker.getPhysicalPlan() is called {}", retryAttempts);
186186
return getQueryPlan(context, sql, textPlan);
187187
} catch (Exception e) {
188188
Throwable rootCause = Throwables.getRootCause(e);
@@ -224,20 +224,20 @@ private static PhysicalPlan getPhysicalPlan(QueryContext context, String sql, Po
224224
* @param textPlan text plan
225225
* @return query physical plan
226226
*/
227-
227+
228228
private static ConcurrentMap<QueryPlanCacheKey, PhysicalPlan> getQueryPlanCache = new ConcurrentHashMap<>();
229-
229+
230230
private static PhysicalPlan getQueryPlan(QueryContext context, String sql, Pointer<String> textPlan)
231231
throws ForemanSetupException, RelConversionException, IOException, ValidationException {
232232

233233
final SqlConverter parser = new SqlConverter(context);
234234
injector.injectChecked(context.getExecutionControls(), "sql-parsing", ForemanSetupException.class);
235235
final SqlNode sqlNode = checkAndApplyAutoLimit(parser, context, sql);
236236
QueryPlanCacheKey queryPlanCacheKey = new QueryPlanCacheKey(sqlNode);
237-
237+
238238
if(getQueryPlanCache.containsKey(queryPlanCacheKey)) {
239-
logger.info("Using getQueryPlanCache");
240-
return getQueryPlanCache.get(queryPlanCacheKey);
239+
logger.info("Using getQueryPlanCache");
240+
return getQueryPlanCache.get(queryPlanCacheKey);
241241
}
242242
final AbstractSqlHandler handler;
243243
final SqlHandlerConfig config = new SqlHandlerConfig(context, parser);
@@ -303,8 +303,8 @@ private static PhysicalPlan getQueryPlan(QueryContext context, String sql, Point
303303
handler = new DefaultSqlHandler(config, textPlan);
304304
context.setSQLStatementType(SqlStatementType.OTHER);
305305
}
306-
307-
306+
307+
308308

309309
// Determines whether result set should be returned for the query based on return result set option and sql node kind.
310310
// Overrides the option on a query level if it differs from the current value.
@@ -318,7 +318,7 @@ private static PhysicalPlan getQueryPlan(QueryContext context, String sql, Point
318318
getQueryPlanCache.put(queryPlanCacheKey, physicalPlan);
319319
return physicalPlan;
320320
}
321-
321+
322322
private static class QueryPlanCacheKey {
323323
private final SqlNode sqlNode;
324324

@@ -328,17 +328,20 @@ public QueryPlanCacheKey(SqlNode sqlNode) {
328328

329329
@Override
330330
public boolean equals(Object o) {
331-
if (this == o) return true;
332-
if (o == null || getClass() != o.getClass()) return false;
331+
if (this == o) {
332+
return true;
333+
}
334+
if (o == null || getClass() != o.getClass()) {
335+
return false;
336+
}
333337
QueryPlanCacheKey cacheKey = (QueryPlanCacheKey) o;
334338
return sqlNode.equalsDeep(cacheKey.sqlNode, Litmus.IGNORE);
335339
}
336-
337-
@Override
338-
public int hashCode() {
339-
return Objects.hash(sqlNode);
340-
}
341340

341+
@Override
342+
public int hashCode() {
343+
return Objects.hash(sqlNode);
344+
}
342345
}
343346

344347
private static boolean isAutoLimitShouldBeApplied(SqlNode sqlNode, int queryMaxRows) {

exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java

Lines changed: 15 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,6 @@
2121
import java.util.Collection;
2222
import java.util.List;
2323
import java.util.Objects;
24-
import java.util.concurrent.ConcurrentHashMap;
25-
import java.util.concurrent.ConcurrentMap;
2624
import java.util.concurrent.TimeUnit;
2725
import java.util.stream.Collectors;
2826

@@ -383,22 +381,26 @@ public CacheKey(PlannerType plannerType, PlannerPhase phase, RelNode input, RelT
383381

384382
@Override
385383
public boolean equals(Object o) {
386-
if (this == o) return true;
387-
if (o == null || getClass() != o.getClass()) return false;
384+
if (this == o) {
385+
return true;
386+
}
387+
if (o == null || getClass() != o.getClass()) {
388+
return false;
389+
}
388390
CacheKey cacheKey = (CacheKey) o;
389-
logger.info("Compare phase {} {} ,{} ", phase.equals(cacheKey.phase), phase.name(), cacheKey.phase.name());
390-
logger.info("Compare plannerType {} {} {}", plannerType.equals(cacheKey.plannerType), plannerType.name() , cacheKey.plannerType.name());
391+
logger.info("Compare phase {} {}, {} ", phase.equals(cacheKey.phase), phase.name(), cacheKey.phase.name());
392+
logger.info("Compare plannerType {} {} {}", plannerType.equals(cacheKey.plannerType), plannerType.name(), cacheKey.plannerType.name());
391393
logger.info("Compare input {}", input.deepEquals(cacheKey.input));
392394
return phase.name().equals(cacheKey.phase.name()) &&
393-
plannerType.name().equals(cacheKey.plannerType.name()) &&
394-
input.deepEquals(cacheKey.input) &&
395-
targetTraits.equals(cacheKey.targetTraits);
395+
plannerType.name().equals(cacheKey.plannerType.name()) &&
396+
input.deepEquals(cacheKey.input) &&
397+
targetTraits.equals(cacheKey.targetTraits);
396398
}
397399

398400
@Override
399-
public int hashCode() {
400-
return Objects.hash(phase.name(), plannerType.name(), input.deepHashCode(), targetTraits);
401-
}
401+
public int hashCode() {
402+
return Objects.hash(phase.name(), plannerType.name(), input.deepHashCode(), targetTraits);
403+
}
402404

403405
}
404406

@@ -422,7 +424,7 @@ protected RelNode transform(PlannerType plannerType, PlannerPhase phase, RelNode
422424
switch (plannerType) {
423425
case HEP_BOTTOM_UP:
424426
case HEP: {
425-
logger.info("DefaultSqlHandler.transform()");
427+
logger.info("DefaultSqlHandler.transform()");
426428
final HepProgramBuilder hepPgmBldr = new HepProgramBuilder();
427429
if (plannerType == PlannerType.HEP_BOTTOM_UP) {
428430
hepPgmBldr.addMatchOrder(HepMatchOrder.BOTTOM_UP);

exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java

Lines changed: 17 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -415,7 +415,7 @@ private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupExcep
415415
}
416416

417417
private void runPhysicalPlan(final PhysicalPlan plan, Pointer<String> textPlan) throws ExecutionSetupException {
418-
validatePlan(plan);
418+
validatePlan(plan);
419419

420420
queryRM.visitAbstractPlan(plan);
421421
final QueryWorkUnit work = getQueryWorkUnit(plan, queryRM);
@@ -486,7 +486,7 @@ private void runFragment(List<PlanFragment> fragmentsList) throws ExecutionSetup
486486
* Moves query to RUNNING state.
487487
*/
488488
private void startQueryProcessing() {
489-
logger.info("Starting query processing");
489+
logger.info("Starting query processing");
490490
enqueue();
491491
runFragments();
492492
queryStateProcessor.moveToState(QueryState.RUNNING, null);
@@ -595,25 +595,25 @@ private void logWorkUnit(QueryWorkUnit queryWorkUnit) {
595595
queryId, queryWorkUnit.stringifyFragments()));
596596
}
597597

598-
private void runSQL(final String sql) throws ExecutionSetupException {
599-
final Pointer<String> textPlan = new Pointer<>();
598+
private void runSQL(final String sql) throws ExecutionSetupException {
599+
final Pointer<String> textPlan = new Pointer<>();
600600

601-
PhysicalPlan plan = CustomCacheManager.getQueryPlan(sql);
601+
PhysicalPlan plan = CustomCacheManager.getQueryPlan(sql);
602602

603-
if (plan == null) {
604-
logger.info("Cache miss, generating new plan");
605-
plan = DrillSqlWorker.getPlan(queryContext, sql, textPlan);
606-
} else {
607-
logger.info("Using cached plan");
608-
}
603+
if (plan == null) {
604+
logger.info("Cache miss, generating new plan");
605+
plan = DrillSqlWorker.getPlan(queryContext, sql, textPlan);
606+
} else {
607+
logger.info("Using cached plan");
608+
}
609609

610-
if(sql.trim().startsWith("SELECT")) {
611-
CustomCacheManager.putQueryPlan(sql, plan);
612-
CustomCacheManager.logCacheStats();
613-
}
610+
if(sql.trim().startsWith("SELECT")) {
611+
CustomCacheManager.putQueryPlan(sql, plan);
612+
CustomCacheManager.logCacheStats();
613+
}
614614

615-
runPhysicalPlan(plan, textPlan);
616-
}
615+
runPhysicalPlan(plan, textPlan);
616+
}
617617

618618
private PhysicalPlan convert(final LogicalPlan plan) throws OptimizerException {
619619
if (logger.isDebugEnabled()) {

0 commit comments

Comments
 (0)