You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2019/01/04 06:54:07 UTC

[GitHub] gparai closed pull request #1572: DRILL-6879: Show warnings for potential performance issues

gparai closed pull request #1572: DRILL-6879: Show warnings for potential performance issues
URL: https://github.com/apache/drill/pull/1572
 
 
   

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

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

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index afa29d902e3..c60426d9e22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -783,6 +783,17 @@ private ExecConstants() {
   public static final BooleanValidator DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR = new BooleanValidator(DYNAMIC_UDF_SUPPORT_ENABLED,
       new OptionDescription("Enables users to dynamically upload UDFs. Users must upload their UDF (source and binary) JAR files to a staging directory in the distributed file system before issuing the CREATE FUNCTION USING JAR command to register a UDF. Default is true. (Drill 1.9+)"));
 
+  //Trigger warning in UX if fragments appear to be doing no work (units are in seconds).
+  public static final String PROFILE_WARNING_PROGRESS_THRESHOLD = "drill.exec.http.profile.warning.progress.threshold";
+  //Trigger warning in UX if slowest fragment operator crosses min threshold and exceeds ratio with average (units are in seconds).
+  public static final String PROFILE_WARNING_TIME_SKEW_MIN = "drill.exec.http.profile.warning.time.skew.min";
+  //Threshold Ratio for Processing (i.e. "maxProcessing : avgProcessing" ratio must exceed this defined threshold to show a skew warning)
+  public static final String PROFILE_WARNING_TIME_SKEW_RATIO_PROCESS = "drill.exec.http.profile.warning.time.skew.ratio.process";
+  //Trigger warning in UX if slowest fragment SCAN crosses min threshold and exceeds ratio with average (units are in seconds).
+  public static final String PROFILE_WARNING_SCAN_WAIT_MIN = "drill.exec.http.profile.warning.scan.wait.min";
+  //Threshold Ratio for Waiting (i.e. "maxWait : avgWait" ratio must exceed this defined threshold to show a skew warning)
+  public static final String PROFILE_WARNING_TIME_SKEW_RATIO_WAIT = "drill.exec.http.profile.warning.time.skew.ratio.wait";
+
   /**
    * Option to save query profiles. If false, no query profile will be saved
    * for any query.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/FragmentWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/FragmentWrapper.java
index d8c5ecbb31d..aead9e0a435 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/FragmentWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/FragmentWrapper.java
@@ -22,12 +22,14 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
 import org.apache.drill.exec.proto.UserBitShared.StreamProfile;
-
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Collections2;
 
@@ -37,10 +39,13 @@
 public class FragmentWrapper {
   private final MajorFragmentProfile major;
   private final long start;
+  private final int runningProfileProgressThreshold;
 
-  public FragmentWrapper(final MajorFragmentProfile major, final long start) {
+  public FragmentWrapper(final MajorFragmentProfile major, final long start, DrillConfig config) {
     this.major = Preconditions.checkNotNull(major);
     this.start = start;
+    //Threshold to track if query made no progress in specified elapsed time
+    runningProfileProgressThreshold = config.getInt(ExecConstants.PROFILE_WARNING_PROGRESS_THRESHOLD);
   }
 
   public String getDisplayName() {
@@ -83,7 +88,7 @@ public void addSummary(TableBuilder tb) {
 
     // If there are no stats to aggregate, create an empty row
     if (complete.size() < 1) {
-      tb.appendRepeated("", null, NUM_NULLABLE_ACTIVE_OVERVIEW_COLUMNS);
+      tb.appendRepeated("", NUM_NULLABLE_ACTIVE_OVERVIEW_COLUMNS);
       return;
     }
 
@@ -118,15 +123,23 @@ public void addSummary(TableBuilder tb) {
     tb.appendMillis(cumulativeFragmentDurationInMillis / complete.size());
     tb.appendMillis(longRun.getEndTime() - longRun.getStartTime());
 
-    tb.appendPercent(totalProcessInMillis / (totalProcessInMillis + totalWaitInMillis), null,
-        //#8721 is the summation sign: sum(Busy): ## + sum(Wait): ##
+    Map<String, String> percBusyAttrMap = new HashMap<>();
+    //#8721 is the summation sign: sum(Busy): ## + sum(Wait): ##
+    percBusyAttrMap.put(HtmlAttribute.TITLE,
         String.format("&#8721;Busy: %,.2fs + &#8721;Wait: %,.2fs", totalProcessInMillis/1E3, totalWaitInMillis/1E3));
+    tb.appendPercent(totalProcessInMillis / (totalProcessInMillis + totalWaitInMillis), percBusyAttrMap);
 
     final MinorFragmentProfile lastUpdate = Collections.max(complete, Comparators.lastUpdate);
     tb.appendMillis(System.currentTimeMillis()-lastUpdate.getLastUpdate());
 
     final MinorFragmentProfile lastProgress = Collections.max(complete, Comparators.lastProgress);
-    tb.appendMillis(System.currentTimeMillis()-lastProgress.getLastProgress());
+    long elapsedSinceLastProgress = System.currentTimeMillis()-lastProgress.getLastProgress();
+    Map<String, String> lastProgressAttrMap = null;
+    if (elapsedSinceLastProgress > TimeUnit.SECONDS.toMillis(runningProfileProgressThreshold)) {
+      lastProgressAttrMap = new HashMap<>();
+      lastProgressAttrMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_NO_PROGRESS_TAG);
+    }
+    tb.appendMillis(elapsedSinceLastProgress, lastProgressAttrMap);
 
     // TODO(DRILL-3494): Names (maxMem, getMaxMemoryUsed) are misleading; the value is peak memory allocated to fragment
     final MinorFragmentProfile maxMem = Collections.max(complete, Comparators.fragmentPeakMemory);
@@ -162,7 +175,7 @@ public void addFinalSummary(TableBuilder tb) {
 
     // If there are no stats to aggregate, create an empty row
     if (complete.size() < 1) {
-      tb.appendRepeated("", null, NUM_NULLABLE_COMPLETED_OVERVIEW_COLUMNS);
+      tb.appendRepeated("", NUM_NULLABLE_COMPLETED_OVERVIEW_COLUMNS);
       return;
     }
 
@@ -195,9 +208,11 @@ public void addFinalSummary(TableBuilder tb) {
     tb.appendMillis(totalDuration / complete.size());
     tb.appendMillis(longRun.getEndTime() - longRun.getStartTime());
 
-    tb.appendPercent(totalProcessInMillis / (totalProcessInMillis + totalWaitInMillis), null,
-        //#8721 is the summation sign: sum(Busy): ## + sum(Wait): ##
+    Map<String, String> percBusyAttrMap = new HashMap<>();
+    //#8721 is the summation sign: sum(Busy): ## + sum(Wait): ##
+    percBusyAttrMap.put(HtmlAttribute.TITLE,
         String.format("&#8721;Busy: %,.2fs + &#8721;Wait: %,.2fs", totalProcessInMillis/1E3, totalWaitInMillis/1E3));
+    tb.appendPercent(totalProcessInMillis / (totalProcessInMillis + totalWaitInMillis), percBusyAttrMap);
 
     // TODO(DRILL-3494): Names (maxMem, getMaxMemoryUsed) are misleading; the value is peak memory allocated to fragment
     final MinorFragmentProfile maxMem = Collections.max(complete, Comparators.fragmentPeakMemory);
@@ -231,9 +246,9 @@ public String getContent() {
 
     Collections.sort(complete, Comparators.minorId);
 
-    Map<String, String> attributeMap = new HashMap<String, String>(); //Reusing for different fragments
+    Map<String, String> attributeMap = new HashMap<>(); //Reusing for different fragments
     for (final MinorFragmentProfile minor : complete) {
-      final ArrayList<OperatorProfile> ops = new ArrayList<>(minor.getOperatorProfileList());
+      final List<OperatorProfile> ops = new ArrayList<>(minor.getOperatorProfileList());
 
       long biggestIncomingRecords = 0;
       long biggestBatches = 0;
@@ -267,7 +282,7 @@ public String getContent() {
 
     for (final MinorFragmentProfile m : incomplete) {
       builder.appendCell(major.getMajorFragmentId() + "-" + m.getMinorFragmentId());
-      builder.appendRepeated(m.getState().toString(), null, NUM_NULLABLE_FRAGMENTS_COLUMNS);
+      builder.appendRepeated(m.getState().toString(), NUM_NULLABLE_FRAGMENTS_COLUMNS);
     }
     return builder.build();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/HtmlAttribute.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/HtmlAttribute.java
new file mode 100644
index 00000000000..75db2988aee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/HtmlAttribute.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.server.rest.profile;
+
+/**
+ * Define all attributes and values that can be injected by various Wrapper classes in org.apache.drill.exec.server.rest.*
+ */
+public class HtmlAttribute {
+  //Attributes
+  public static final String CLASS = "class";
+  public static final String DATA_ORDER = "data-order";
+  public static final String TITLE = "title";
+  public static final String SPILLS = "spills";
+  public static final String STYLE = "style";
+
+  //Values
+  public static final String CLASS_VALUE_SPILL_TAG = "spill-tag";
+  public static final String CLASS_VALUE_NO_PROGRESS_TAG = "no-progress-tag";
+  public static final String CLASS_VALUE_TIME_SKEW_TAG = "time-skew-tag";
+  public static final String CLASS_VALUE_SCAN_WAIT_TAG = "scan-wait-tag";
+  public static final String STYLE_VALUE_CURSOR_HELP = "cursor:help;";
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
index f6803bf80c4..0f611708acb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
@@ -25,14 +25,16 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.OperatorMetricRegistry;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.proto.UserBitShared.MetricValue;
 import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
 import org.apache.drill.exec.proto.UserBitShared.StreamProfile;
-
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
 /**
@@ -42,10 +44,6 @@
   @SuppressWarnings("unused")
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorWrapper.class);
 
-  private static final String HTML_ATTRIB_SPILLS = "spills";
-  private static final String HTML_ATTRIB_CLASS = "class";
-  private static final String HTML_ATTRIB_STYLE = "style";
-  private static final String HTML_ATTRIB_TITLE = "title";
   private static final DecimalFormat DECIMAL_FORMATTER = new DecimalFormat("#.##");
   private static final String UNKNOWN_OPERATOR = "UNKNOWN_OPERATOR";
   //Negative valued constant used for denoting invalid index to indicate absence of metric
@@ -56,8 +54,19 @@
   private final CoreOperatorType operatorType;
   private final String operatorName;
   private final int size;
+  private final int timeSkewMin;
+  private final double timeSkewRatio;
+  private final int scanWaitMin;
+  private final double waitSkewRatio;
+
+  public OperatorWrapper(int major, List<ImmutablePair<ImmutablePair<OperatorProfile, Integer>, String>> opsAndHostsList, Map<String, String> phyOperMap, DrillConfig config) {
+    //Threshold to track if the slowest operator ran relatively slow
+    timeSkewMin = config.getInt(ExecConstants.PROFILE_WARNING_TIME_SKEW_MIN);
+    timeSkewRatio = config.getDouble(ExecConstants.PROFILE_WARNING_TIME_SKEW_RATIO_PROCESS);
+    //Threshold to track if the slowest SCAN operator spent more time in wait than processing
+    scanWaitMin = config.getInt(ExecConstants.PROFILE_WARNING_SCAN_WAIT_MIN);
+    waitSkewRatio = config.getDouble(ExecConstants.PROFILE_WARNING_TIME_SKEW_RATIO_WAIT);
 
-  public OperatorWrapper(int major, List<ImmutablePair<ImmutablePair<OperatorProfile, Integer>, String>> opsAndHostsList, Map<String, String> phyOperMap) {
     Preconditions.checkArgument(opsAndHostsList.size() > 0);
     this.major = major;
     firstProfile = opsAndHostsList.get(0).getLeft().getLeft();
@@ -102,12 +111,12 @@ public String getId() {
   public String getContent() {
     TableBuilder builder = new TableBuilder(OPERATOR_COLUMNS, OPERATOR_COLUMNS_TOOLTIP, true);
 
-    Map<String, String> attributeMap = new HashMap<String, String>(); //Reusing for different fragments
+    Map<String, String> attributeMap = new HashMap<>(); //Reusing for different fragments
     for (ImmutablePair<ImmutablePair<OperatorProfile, Integer>, String> ip : opsAndHosts) {
       int minor = ip.getLeft().getRight();
       OperatorProfile op = ip.getLeft().getLeft();
 
-      attributeMap.put("data-order", String.valueOf(minor)); //Overwrite values from previous fragments
+      attributeMap.put(HtmlAttribute.DATA_ORDER, String.valueOf(minor)); //Overwrite values from previous fragments
       String path = new OperatorPathBuilder().setMajor(major).setMinor(minor).setOperator(op).build();
       builder.appendCell(path, attributeMap);
       builder.appendCell(ip.getRight());
@@ -150,17 +159,18 @@ public String getContent() {
   //Palette to help shade operators sharing a common major fragment
   private static final String[] OPERATOR_OVERVIEW_BGCOLOR_PALETTE = {"#ffffff","#f2f2f2"};
 
-  public void addSummary(TableBuilder tb, HashMap<String, Long> majorFragmentBusyTally, long majorFragmentBusyTallyTotal) {
+  public void addSummary(TableBuilder tb, Map<String, Long> majorFragmentBusyTally, long majorFragmentBusyTallyTotal) {
     //Select background color from palette
     String opTblBgColor = OPERATOR_OVERVIEW_BGCOLOR_PALETTE[major%OPERATOR_OVERVIEW_BGCOLOR_PALETTE.length];
     String path = new OperatorPathBuilder().setMajor(major).setOperator(firstProfile).build();
-    tb.appendCell(path, null, null, opTblBgColor);
+    tb.appendCell(path, opTblBgColor, null);
     tb.appendCell(operatorName);
 
     //Check if spill information is available
     int spillCycleMetricIndex = getSpillCycleMetricIndex(operatorType);
     boolean isSpillableOp = (spillCycleMetricIndex != NO_SPILL_METRIC_INDEX);
     boolean hasSpilledToDisk = false;
+    boolean isScanOp = operatorName.endsWith("SCAN");
 
     //Get MajorFragment Busy+Wait Time Tally
     long majorBusyNanos = majorFragmentBusyTally.get(new OperatorPathBuilder().setMajor(major).build());
@@ -208,15 +218,53 @@ public void addSummary(TableBuilder tb, HashMap<String, Long> majorFragmentBusyT
     tb.appendNanos(Math.round(setupSum / size));
     tb.appendNanos(longSetup.getLeft().getSetupNanos());
 
+    Map<String, String> timeSkewMap = null;
     final ImmutablePair<OperatorProfile, Integer> longProcess = Collections.max(opList, Comparators.processTime);
-    tb.appendNanos(Math.round(processSum / size));
-    tb.appendNanos(longProcess.getLeft().getProcessNanos());
+    //Calculating average processing time
+    long avgProcTime = Math.round(processSum / size);
+    tb.appendNanos(avgProcTime);
+    long maxProcTime = longProcess.getLeft().getProcessNanos();
+    //Calculating skew of longest processing fragment w.r.t. average
+    double maxSkew = (avgProcTime > 0) ? maxProcTime/Double.valueOf(avgProcTime) : 0.0d;
+    //Marking skew if both thresholds are crossed
+    if (avgProcTime > TimeUnit.SECONDS.toNanos(timeSkewMin) && maxSkew > timeSkewRatio ) {
+      timeSkewMap = new HashMap<>();
+      timeSkewMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_TIME_SKEW_TAG);
+      timeSkewMap.put(HtmlAttribute.TITLE,  "One fragment took " + DECIMAL_FORMATTER.format(maxSkew) + " longer than average");
+      timeSkewMap.put(HtmlAttribute.STYLE, HtmlAttribute.STYLE_VALUE_CURSOR_HELP);
+    }
+    tb.appendNanos(maxProcTime, timeSkewMap);
 
     final ImmutablePair<OperatorProfile, Integer> shortWait = Collections.min(opList, Comparators.waitTime);
     final ImmutablePair<OperatorProfile, Integer> longWait = Collections.max(opList, Comparators.waitTime);
     tb.appendNanos(shortWait.getLeft().getWaitNanos());
-    tb.appendNanos(Math.round(waitSum / size));
-    tb.appendNanos(longWait.getLeft().getWaitNanos());
+    //Calculating average wait time for fragment
+    long avgWaitTime = Math.round(waitSum / size);
+
+    //Slow Scan Warning
+    Map<String, String> slowScanMap = null;
+    //Marking slow scan if threshold is crossed and wait was longer than processing
+    if (isScanOp && (avgWaitTime > TimeUnit.SECONDS.toNanos(scanWaitMin)) && (avgWaitTime > avgProcTime)) {
+      slowScanMap = new HashMap<>();
+      slowScanMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_SCAN_WAIT_TAG);
+      slowScanMap.put(HtmlAttribute.TITLE, "Avg Wait Time &gt; Avg Processing Time");
+      slowScanMap.put(HtmlAttribute.STYLE, HtmlAttribute.STYLE_VALUE_CURSOR_HELP);
+    }
+    tb.appendNanos(avgWaitTime, slowScanMap);
+
+    long maxWaitTime = longWait.getLeft().getWaitNanos();
+    //Skewed Wait Warning
+    timeSkewMap = null; //Resetting
+    //Calculating skew of longest waiting fragment w.r.t. average
+    maxSkew = (avgWaitTime > 0) ? maxWaitTime/Double.valueOf(avgWaitTime) : 0.0d;
+    //Marking skew if both thresholds are crossed
+    if (avgWaitTime > TimeUnit.SECONDS.toNanos(timeSkewMin) && maxSkew > waitSkewRatio) {
+      timeSkewMap = new HashMap<>();
+      timeSkewMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_TIME_SKEW_TAG);
+      timeSkewMap.put(HtmlAttribute.TITLE, "One fragment waited " + DECIMAL_FORMATTER.format(maxSkew) + " longer than average");
+      timeSkewMap.put(HtmlAttribute.STYLE, HtmlAttribute.STYLE_VALUE_CURSOR_HELP);
+    }
+    tb.appendNanos(maxWaitTime, timeSkewMap);
 
     tb.appendPercent(processSum / majorBusyNanos);
     tb.appendPercent(processSum / majorFragmentBusyTallyTotal);
@@ -232,15 +280,15 @@ public void addSummary(TableBuilder tb, HashMap<String, Long> majorFragmentBusyT
       avgSpillMap = new HashMap<>();
       //Average SpillCycle
       double avgSpillCycle = spillCycleSum/size;
-      avgSpillMap.put(HTML_ATTRIB_TITLE, DECIMAL_FORMATTER.format(avgSpillCycle) + " spills on average");
-      avgSpillMap.put(HTML_ATTRIB_STYLE, "cursor:help;" + spillCycleMax);
-      avgSpillMap.put(HTML_ATTRIB_CLASS, "spill-tag"); //JScript will inject Icon
-      avgSpillMap.put(HTML_ATTRIB_SPILLS, DECIMAL_FORMATTER.format(avgSpillCycle)); //JScript will inject Count
+      avgSpillMap.put(HtmlAttribute.TITLE, DECIMAL_FORMATTER.format(avgSpillCycle) + " spills on average");
+      avgSpillMap.put(HtmlAttribute.STYLE, HtmlAttribute.STYLE_VALUE_CURSOR_HELP);
+      avgSpillMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_SPILL_TAG); //JScript will inject Icon
+      avgSpillMap.put(HtmlAttribute.SPILLS, DECIMAL_FORMATTER.format(avgSpillCycle)); //JScript will inject Count
       maxSpillMap = new HashMap<>();
-      maxSpillMap.put(HTML_ATTRIB_TITLE, "Most # spills: " + spillCycleMax);
-      maxSpillMap.put(HTML_ATTRIB_STYLE, "cursor:help;" + spillCycleMax);
-      maxSpillMap.put(HTML_ATTRIB_CLASS, "spill-tag"); //JScript will inject Icon
-      maxSpillMap.put(HTML_ATTRIB_SPILLS, String.valueOf(spillCycleMax)); //JScript will inject Count
+      maxSpillMap.put(HtmlAttribute.TITLE, "Most # spills: " + spillCycleMax);
+      maxSpillMap.put(HtmlAttribute.STYLE, HtmlAttribute.STYLE_VALUE_CURSOR_HELP);
+      maxSpillMap.put(HtmlAttribute.CLASS, HtmlAttribute.CLASS_VALUE_SPILL_TAG); //JScript will inject Icon
+      maxSpillMap.put(HtmlAttribute.SPILLS, String.valueOf(spillCycleMax)); //JScript will inject Count
     }
 
     tb.appendBytes(Math.round(memSum / size), avgSpillMap);
@@ -312,7 +360,7 @@ public String getMetricsTable() {
 
       final Number[] values = new Number[metricNames.length];
       //Track new/Unknown Metrics
-      final Set<Integer> unknownMetrics = new TreeSet<Integer>();
+      final Set<Integer> unknownMetrics = new TreeSet<>();
       for (final MetricValue metric : op.getMetricList()) {
         if (metric.getMetricId() < metricNames.length) {
           if (metric.hasLongValue()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
index 506900dee30..af2b7905a08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
@@ -201,10 +201,10 @@ protected ClusterCoordinator getCoordinator() {
     private List<ProfileInfo> finishedQueries;
     private List<String> errors;
 
-    public QProfiles(List<ProfileInfo> runningQueries, List<ProfileInfo> finishedQueries, List<String> erorrs) {
+    public QProfiles(List<ProfileInfo> runningQueries, List<ProfileInfo> finishedQueries, List<String> errors) {
       this.runningQueries = runningQueries;
       this.finishedQueries = finishedQueries;
-      this.errors = erorrs;
+      this.errors = errors;
     }
 
     public List<ProfileInfo> getRunningQueries() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
index 35a37065597..7e72556bc80 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
@@ -30,6 +30,7 @@
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
@@ -40,9 +41,9 @@
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.rest.WebServer;
+import org.apache.drill.shaded.guava.com.google.common.base.CaseFormat;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.shaded.guava.com.google.common.base.CaseFormat;
 
 /**
  * Wrapper class for a {@link #profile query profile}, so it to be presented through web UI.
@@ -57,11 +58,12 @@
   private final String id;
   private final List<FragmentWrapper> fragmentProfiles;
   private final List<OperatorWrapper> operatorProfiles;
-  private final HashMap<String, Long> majorFragmentTallyMap;
+  private final Map<String, Long> majorFragmentTallyMap;
   private final long majorFragmentTallyTotal;
   private final OptionList options;
   private final boolean onlyImpersonationEnabled;
   private Map<String, String> physicalOperatorMap;
+  private final String noProgressWarningThreshold;
 
   public ProfileWrapper(final QueryProfile profile, DrillConfig drillConfig) {
     this.profile = profile;
@@ -76,7 +78,7 @@ public ProfileWrapper(final QueryProfile profile, DrillConfig drillConfig) {
     Collections.sort(majors, Comparators.majorId);
 
     for (final MajorFragmentProfile major : majors) {
-      fragmentProfiles.add(new FragmentWrapper(major, profile.getStart()));
+      fragmentProfiles.add(new FragmentWrapper(major, profile.getStart(), drillConfig));
     }
     this.fragmentProfiles = fragmentProfiles;
     this.majorFragmentTallyMap = new HashMap<>(majors.size());
@@ -115,7 +117,7 @@ public ProfileWrapper(final QueryProfile profile, DrillConfig drillConfig) {
     Collections.sort(keys);
 
     for (final ImmutablePair<Integer, Integer> ip : keys) {
-      ows.add(new OperatorWrapper(ip.getLeft(), opmap.get(ip), physicalOperatorMap));
+      ows.add(new OperatorWrapper(ip.getLeft(), opmap.get(ip), physicalOperatorMap, drillConfig));
     }
     this.operatorProfiles = ows;
 
@@ -129,6 +131,7 @@ public ProfileWrapper(final QueryProfile profile, DrillConfig drillConfig) {
     this.options = options;
 
     this.onlyImpersonationEnabled = WebServer.isImpersonationOnlyEnabled(drillConfig);
+    this.noProgressWarningThreshold = String.valueOf(drillConfig.getInt(ExecConstants.PROFILE_WARNING_PROGRESS_THRESHOLD));
   }
 
   private long tallyMajorFragmentCost(List<MajorFragmentProfile> majorFragments) {
@@ -260,6 +263,11 @@ public String getExecutionDuration() {
     return NOT_AVAILABLE_LABEL;
   }
 
+  //Threshold to be used by WebServer in issuing warning
+  public String getNoProgressWarningThreshold() {
+    return this.noProgressWarningThreshold;
+  }
+
   public List<FragmentWrapper> getFragmentProfiles() {
     return fragmentProfiles;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/TableBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/TableBuilder.java
index ad89b3f8b62..615af0bf31f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/TableBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/TableBuilder.java
@@ -26,6 +26,7 @@
 import java.util.Map;
 
 public class TableBuilder {
+  private static final String NO_BGCOLOR = "";
   private final NumberFormat format = NumberFormat.getInstance(Locale.US);
   private final DateFormat dateFormat = new SimpleDateFormat("HH:mm:ss");
   private final DecimalFormat dec = new DecimalFormat("0.00");
@@ -45,10 +46,10 @@ public TableBuilder(final String[] columns, final String[] columnTooltip, final
 
     format.setMaximumFractionDigits(3);
 
-    sb.append("<table class=\"table table-bordered text-right"+(isSortable? " sortable" : "")+"\">\n<thead><tr>");
+    sb.append("<table class=\"table table-bordered text-right"+(isSortable? " sortable" : NO_BGCOLOR)+"\">\n<thead><tr>");
     for (int i = 0; i < columns.length; i++) {
       String cn = columns[i];
-      String ctt = "";
+      String ctt = NO_BGCOLOR;
       if (columnTooltip != null) {
         String tooltip = columnTooltip[i];
         if (tooltip != null) {
@@ -61,37 +62,26 @@ public TableBuilder(final String[] columns, final String[] columnTooltip, final
   }
 
   public void appendCell(final String s) {
-    appendCell(s, null, null, null);
+    appendCell(s, NO_BGCOLOR, null);
   }
 
-  public void appendCell(final String s, final String link) {
-    appendCell(s, link, null, null);
-  }
-
-  public void appendCell(final String s, final String link, final String titleText) {
-    appendCell(s, link, titleText, null);
-  }
-
-  public void appendCell(final String s, final String link, final String titleText, final String backgroundColor) {
-    appendCell(s, link, titleText, backgroundColor, null);
+  public void appendCell(final String s, final String backgroundColor) {
+    appendCell(s, backgroundColor, null);
   }
 
   public void appendCell(final String s, final Map<String, String> kvPairs) {
-    appendCell(s, null, null, null, kvPairs);
+    appendCell(s, NO_BGCOLOR, kvPairs);
   }
 
-  public void appendCell(final String s, final String link, final String titleText, final String backgroundColor,
-      final Map<String, String> kvPairs) {
+  //Inject value into a table cell. Start or end a row if required
+  public void appendCell(final String s, final String rowBackgroundColor, final Map<String, String> kvPairs) {
+    //Check if this is first column?
     if (w == 0) {
       sb.append("<tr"
-          + (backgroundColor == null ? "" : " style=\"background-color:"+backgroundColor+"\"")
+          + (rowBackgroundColor == null || rowBackgroundColor == NO_BGCOLOR ? "" : " style=\"background-color:"+rowBackgroundColor+"\"")
           + ">");
     }
     StringBuilder tdElemSB = new StringBuilder("<td");
-    //Injecting title if specified (legacy impl)
-    if (titleText != null && titleText.length() > 0) {
-      tdElemSB.append(" title=\""+titleText+"\"");
-    }
     //Extract other attributes for injection into element
     if (kvPairs != null) {
       for (String attributeName : kvPairs.keySet()) {
@@ -99,8 +89,8 @@ public void appendCell(final String s, final String link, final String titleText
         tdElemSB.append(attributeText);
       }
     }
-    //Closing <td>
-    tdElemSB.append(String.format(">%s%s</td>", s, link != null ? link : ""));
+    //Inserting inner text value and closing <td>
+    tdElemSB.append(">").append(s).append("</td>");
     sb.append(tdElemSB);
     if (++w >= width) {
       sb.append("</tr>\n");
@@ -108,13 +98,14 @@ public void appendCell(final String s, final String link, final String titleText
     }
   }
 
-  public void appendRepeated(final String s, final String link, final int n) {
-    appendRepeated(s, link, n, null);
+  public void appendRepeated(final String s, final int n) {
+    appendRepeated(s, n, null);
   }
 
-  public void appendRepeated(final String s, final String link, final int n, final String tooltip) {
+  //Inject a value repeatedly into a table cell
+  public void appendRepeated(final String s, final int n, final Map<String, String> attributeMap) {
     for (int i = 0; i < n; i++) {
-      appendCell(s, link, tooltip);
+      appendCell(s, attributeMap);
     }
   }
 
@@ -122,109 +113,82 @@ public void appendTime(final long d) {
     appendTime(d, null);
   }
 
-  public void appendTime(final long d, final String link) {
-    appendTime(d, link, null);
-  }
-
-  public void appendTime(final long d, final String link, final String tooltip) {
+  //Inject timestamp/date value with ordering into a table cell
+  public void appendTime(final long d, Map<String, String> attributeMap) {
     //Embedding dataTable's data-order attribute
-    Map<String, String> attributeMap = new HashMap<String, String>();
-    attributeMap.put("data-order", String.valueOf(d));
-    appendCell(dateFormat.format(d), link, tooltip, null, attributeMap);
+    if (attributeMap == null) {
+      attributeMap = new HashMap<>();
+    }
+    attributeMap.put(HtmlAttribute.DATA_ORDER, String.valueOf(d));
+    appendCell(dateFormat.format(d), null, attributeMap);
   }
 
   public void appendMillis(final long p) {
     appendMillis(p, null);
   }
 
-  public void appendMillis(final long p, final String link) {
-    appendMillis(p, link, null);
-  }
-
-  public void appendMillis(final long p, final String link, final String tooltip) {
+  //Inject millisecond based time value with ordering into a table cell
+  public void appendMillis(final long p, Map<String, String> attributeMap) {
     //Embedding dataTable's data-order attribute
-    Map<String, String> attributeMap = new HashMap<String, String>();
-    attributeMap.put("data-order", String.valueOf(p));
-    appendCell((new SimpleDurationFormat(0, p)).compact(), link, tooltip, null, attributeMap);
+    if (attributeMap == null) {
+      attributeMap = new HashMap<>();
+    }
+    attributeMap.put(HtmlAttribute.DATA_ORDER, String.valueOf(p));
+    appendCell((new SimpleDurationFormat(0, p)).compact(), NO_BGCOLOR, attributeMap);
   }
 
   public void appendNanos(final long p) {
-    appendNanos(p, null, null);
+    appendNanos(p, null);
   }
 
-  public void appendNanos(final long p, final String link) {
-    appendNanos(p, link, null);
-  }
-
-  public void appendNanos(final long p, final String link, final String tooltip) {
-    appendMillis(Math.round(p / 1000.0 / 1000.0), link, tooltip);
+  public void appendNanos(final long p, Map<String, String> attributeMap) {
+    appendMillis(Math.round(p / 1000.0 / 1000.0), attributeMap);
   }
 
   public void appendPercent(final double percentAsFraction) {
-    appendCell(dec.format(100*percentAsFraction).concat("%"), null, null);
-  }
-
-  public void appendPercent(final double percentAsFraction, final String link) {
-    appendCell(dec.format(100*percentAsFraction).concat("%"), link, null);
+    appendCell(dec.format(100*percentAsFraction).concat("%"), NO_BGCOLOR, null);
   }
 
-  public void appendPercent(final double percentAsFraction, final String link, final String tooltip) {
-    appendCell(dec.format(100*percentAsFraction).concat("%"), link, tooltip);
+  //Inject value as a percentage with value between 0 and 100 into a table cell
+  public void appendPercent(final double percentAsFraction, Map<String, String> attributeMap) {
+    appendCell(dec.format(100*percentAsFraction).concat("%"), NO_BGCOLOR, attributeMap);
   }
 
   public void appendFormattedNumber(final Number n) {
-    appendCell(format.format(n), null, null);
+    appendCell(format.format(n), NO_BGCOLOR, null);
   }
 
-  public void appendFormattedNumber(final Number n, final String link) {
-    appendCell(format.format(n), link, null);
-  }
-
-  public void appendFormattedNumber(final Number n, final String link, final String tooltip) {
-    appendCell(format.format(n), link, tooltip);
+  public void appendFormattedNumber(final Number n, Map<String, String> attributeMap) {
+    appendCell(format.format(n), NO_BGCOLOR, attributeMap);
   }
 
   public void appendFormattedInteger(final long n) {
-    appendCell(intformat.format(n), null, null);
-  }
-
-  public void appendFormattedInteger(final long n, final String link) {
-    appendCell(intformat.format(n), link, null);
+    appendCell(intformat.format(n), NO_BGCOLOR, null);
   }
 
-  public void appendFormattedInteger(final long n, final String link, final String tooltip) {
-    appendCell(intformat.format(n), link, tooltip);
+  public void appendFormattedInteger(final long n, Map<String, String> attributeMap) {
+    appendCell(intformat.format(n), NO_BGCOLOR, attributeMap);
   }
 
-  public void appendInteger(final long l, final String link, final String tooltip) {
-    appendCell(Long.toString(l), link, tooltip);
+  public void appendInteger(final long l, Map<String, String> attributeMap) {
+    appendCell(Long.toString(l), NO_BGCOLOR, attributeMap);
   }
 
   public void appendBytes(final long l) {
-    appendBytes(l, null, null, null);
-  }
-
-  public void appendBytes(final long l, final String link) {
-    appendBytes(l, link, null);
-  }
-
-  public void appendBytes(final long l, final String link, final String tooltip) {
-    appendBytes(l, link, tooltip, null);
+    appendBytes(l, null);
   }
 
+  //Inject print-friendly byte value with ordering into a table cell
   public void appendBytes(final long l, Map<String, String> attributeMap) {
-    appendBytes(l, null, null, attributeMap);
-  }
-
-  public void appendBytes(final long l, final String link, final String tooltip, Map<String, String> attributeMap) {
     //Embedding dataTable's data-order attribute
     if (attributeMap == null) {
       attributeMap = new HashMap<>();
     }
-    attributeMap.put("data-order", String.valueOf(l));
-    appendCell(bytePrint(l), link, tooltip, null, attributeMap);
+    attributeMap.put(HtmlAttribute.DATA_ORDER, String.valueOf(l));
+    appendCell(bytePrint(l), NO_BGCOLOR, attributeMap);
   }
 
+  //Generate a print-friendly representation of a byte count
   private String bytePrint(final long size) {
     final double t = size / Math.pow(1024, 4);
     if (t > 1) {
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 3682a855fa0..d3548b894ce 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -142,6 +142,17 @@ drill.exec: {
       }
     }
     max_profiles: 100,
+    profile.warning: {
+      progress.threshold: 300,
+      time.skew: {
+        min: 2,
+        ratio: {
+          process: 2
+          wait: 2
+        }
+      },
+      scan.wait.min: 60
+    },
     session_max_idle_secs: 3600, # Default value 1hr
     cors: {
       enabled: false,
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
index ee126abd3b9..6b4e732d87d 100644
--- a/exec/java-exec/src/main/resources/rest/profile/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -48,8 +48,40 @@
         "lengthChange": false,
         "paging": false,
         "info": false
-      }
-    );} );
+      });
+      //Enable Warnings by making it visible
+      checkForWarnings();
+    });
+
+    //Check for Warnings
+    function checkForWarnings() {
+      //No Progress Warning
+      let noProgressFragmentCount = document.querySelectorAll('td[class=no-progress-tag]').length;
+      let majorFragmentCount = document.querySelectorAll('#fragment-overview table tbody tr').length;
+      toggleWarning("noProgressWarning", majorFragmentCount, noProgressFragmentCount);
+
+      //Spill To Disk Warnings
+      let spillCount = document.querySelectorAll('td[class=spill-tag]').length;
+      toggleWarning("spillToDiskWarning", true, (spillCount > 0));
+
+      //Slow Scan Warnings
+      let longScanWaitCount = document.querySelectorAll('td[class=scan-wait-tag]').length;
+      toggleWarning("longScanWaitWarning", true, (longScanWaitCount > 0));
+    }
+
+    //Show Warnings
+    function toggleWarning(warningElemId, expectedVal, actualVal) {
+        if (expectedVal == actualVal) {
+            document.getElementById(warningElemId).style.display="block";
+        } else {
+            closeWarning(warningElemId);
+        }
+    }
+
+    //Close Warning
+    function closeWarning(warningElemId) {
+        document.getElementById(warningElemId).style.display="none";
+    }
 
     //Close the cancellation status popup
     function refreshStatus() {
@@ -211,7 +243,7 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
   </div>
   </#if>
   </h3>
-  
+
   <div class="panel-group" id="query-profile-accordion">
     <div class="panel panel-default">
       <div class="panel-heading">
@@ -325,26 +357,31 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
       <div id="fragment-overview" class="panel-collapse collapse">
         <div class="panel-body">
           <svg id="fragment-overview-canvas" class="center-block"></svg>
+          <div id="noProgressWarning" style="display:none;cursor:help" class="panel panel-warning">
+            <div class="panel-heading" title="Check if any of the Drillbits are waiting for data from a SCAN operator, or might actually be hung with its VM thread being busy." style="cursor:pointer">
+            <span class="glyphicon glyphicon-alert" style="font-size:125%">&#xe209;</span> <b>WARNING:</b> No fragments have made any progress in the last <b>${model.getNoProgressWarningThreshold()}</b> seconds. (See <span style="font-style:italic;font-weight:bold">Last Progress</span> below)
+            </div>
+          </div>
           ${model.getFragmentsOverview()?no_esc}
         </div>
       </div>
-    </div>
-    <#list model.getFragmentProfiles() as frag>
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        <h4 class="panel-title">
-          <a data-toggle="collapse" href="#${frag.getId()}">
-            ${frag.getDisplayName()}
-          </a>
-        </h4>
-      </div>
-      <div id="${frag.getId()}" class="panel-collapse collapse">
-        <div class="panel-body">
-          ${frag.getContent()?no_esc}
+      <#list model.getFragmentProfiles() as frag>
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          <h4 class="panel-title">
+            <a data-toggle="collapse" href="#${frag.getId()}">
+              ${frag.getDisplayName()}
+            </a>
+          </h4>
+        </div>
+        <div id="${frag.getId()}" class="panel-collapse collapse">
+          <div class="panel-body">
+            ${frag.getContent()?no_esc}
+          </div>
         </div>
       </div>
+      </#list>
     </div>
-    </#list>
   </div>
 
   <div class="page-header"></div>
@@ -361,6 +398,17 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
       </div>
       <div id="operator-overview" class="panel-collapse collapse">
         <div class="panel-body">
+          <div id="spillToDiskWarning" style="display:none;cursor:help" class="panel panel-warning" title="Spills occur because a buffered operator didn't get enough memory to hold data in memory. Increase the memory or ensure that number of spills &lt; 2">
+            <div class="panel-heading"><span class="glyphicon glyphicon-alert" style="font-size:125%">&#xe209;</span> <b>WARNING:</b> Some operators have data spilled to disk. This will result in performance loss. (See <span style="font-style:italic;font-weight:bold">Avg Peak Memory</span> and <span style="font-style:italic;font-weight:bold">Max Peak Memory</span> below)
+            <button type="button" class="close" onclick="closeWarning('spillToDiskWarning')" style="font-size:180%">&times;</button>
+            </div>
+          </div>
+          <div id="longScanWaitWarning" style="display:none;cursor:help" class="panel panel-warning">
+            <div class="panel-heading" title="Check if any of the Drillbits are waiting for data from a SCAN operator, or might actually be hung with its VM thread being busy." style="cursor:pointer">
+            <span class="glyphicon glyphicon-alert" style="font-size:125%">&#xe209;</span> <b>WARNING:</b> Some of the SCAN operators spent more time waiting for the data than processing it. (See <span style="font-style:italic;font-weight:bold">Avg Wait Time</span> as compared to <span style="font-style:italic;font-weight:bold">Average Process Time</span> for the <b>SCAN</b> operators below)
+            <button type="button" class="close" onclick="closeWarning('longScanWaitWarning')" style="font-size:180%">&times;</button>
+            </div>
+          </div>
           ${model.getOperatorsOverview()?no_esc}
         </div>
       </div>
@@ -413,14 +461,33 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
     <script>
     //Inject Spilled Tags
     $(window).on('load', function () {
-      var spillLabel = document.getElementsByClassName("spill-tag");
-      var i;
-      for (i = 0; i < spillLabel.length; i++) {
-        var content = spillLabel[i].innerHTML;
-        spillLabel[i].innerHTML = "<span class=\"glyphicon glyphicon-download-alt\">&nbsp;</span>"+content;
-      }
+      injectIconByClass("spill-tag","glyphicon-download-alt");
+      injectIconByClass("time-skew-tag","glyphicon-time");
+      injectSlowScanIcon();
     });
 
+    //Inject Glyphicon by Class tag
+    function injectIconByClass(tagLabel, tagIcon) {
+        //Inject Spill icons
+        var tagElemList = document.getElementsByClassName(tagLabel);
+        var i;
+        for (i = 0; i < tagElemList.length; i++) {
+            var content = tagElemList[i].innerHTML;
+            tagElemList[i].innerHTML = "<span class=\"glyphicon "+tagIcon+"\">&nbsp;</span>"+content;
+        }
+    }
+
+    //Inject PNG icon for slow
+    function injectSlowScanIcon() {
+        //Inject Spill icons
+        var tagElemList = document.getElementsByClassName("scan-wait-tag");
+        var i;
+        for (i = 0; i < tagElemList.length; i++) {
+            var content = tagElemList[i].innerHTML;
+            tagElemList[i].innerHTML = "<img src='/static/img/turtle.png' alt='slow'> "+content;
+        }
+    }
+
     //Configuration for Query Viewer in Profile
     ace.require("ace/ext/language_tools");
     var viewer = ace.edit("query-text");
diff --git a/exec/java-exec/src/main/resources/rest/static/img/turtle.png b/exec/java-exec/src/main/resources/rest/static/img/turtle.png
new file mode 100644
index 00000000000..ec394a41e64
Binary files /dev/null and b/exec/java-exec/src/main/resources/rest/static/img/turtle.png differ


 

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


With regards,
Apache Git Services