[AMBARI-24723] Support wild cards in AppId and InstanceId fields in AMS GET API. (#5)

* [AMBARI-24723] Support wild cards in AppId and InstanceId fields in AMS GET API.

* [AMBARI-24723] Support wild cards in AppId and InstanceId fields in AMS GET API. (2)

* [AMBARI-24723] Support wild cards in AppId and InstanceId fields in AMS GET API. (3)
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
index c0427e7..b6ff202 100644
--- a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/PhoenixHBaseAccessor.java
@@ -121,6 +121,7 @@
 import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
 import org.apache.ambari.metrics.core.timeline.query.Condition;
 import org.apache.ambari.metrics.core.timeline.query.DefaultPhoenixDataSource;
+import org.apache.ambari.metrics.core.timeline.query.MetadataQueryCondition;
 import org.apache.ambari.metrics.core.timeline.query.PhoenixConnectionProvider;
 import org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL;
 import org.apache.ambari.metrics.core.timeline.query.SplitByMetricNamesCondition;
@@ -1990,6 +1991,106 @@
     return metadataMap;
   }
 
+  public List<TimelineMetricMetadata> scanMetricMetadataForWildCardRequest(Collection<String> metricNames,
+                                                                           String appId,
+                                                                           String instanceId) throws SQLException {
+    List<TimelineMetricMetadata> metadataList = new ArrayList<>();
+    Connection conn = getConnection();
+    PreparedStatement stmt = null;
+    ResultSet rs = null;
+
+    MetadataQueryCondition metadataQueryCondition = new MetadataQueryCondition(new ArrayList<>(metricNames), appId, instanceId);
+    stmt = PhoenixTransactSQL.prepareScanMetricMetadataSqlStmt(conn, metadataQueryCondition);
+    try {
+      if (stmt != null) {
+        rs = stmt.executeQuery();
+        while (rs.next()) {
+          TimelineMetricMetadata metadata = new TimelineMetricMetadata(
+            rs.getString("METRIC_NAME"),
+            rs.getString("APP_ID"),
+            rs.getString("INSTANCE_ID"),
+            null,
+            null,
+            null,
+            false,
+            true
+          );
+
+          metadata.setUuid(rs.getBytes("UUID"));
+          metadataList.add(metadata);
+        }
+      }
+    } finally {
+      if (rs != null) {
+        try {
+          rs.close();
+        } catch (SQLException e) {
+          // Ignore
+        }
+      }
+      if (stmt != null) {
+        try {
+          stmt.close();
+        } catch (SQLException e) {
+          // Ignore
+        }
+      }
+      if (conn != null) {
+        try {
+          conn.close();
+        } catch (SQLException sql) {
+          // Ignore
+        }
+      }
+    }
+
+    return metadataList;
+  }
+
+  public List<byte[]> scanHostMetadataForWildCardRequest(List<String> hostnames) throws SQLException {
+    List<byte[]> uuidList = new ArrayList<>();
+    Connection conn = getConnection();
+    PreparedStatement stmt;
+    ResultSet rs = null;
+
+    MetadataQueryCondition metadataQueryCondition = new MetadataQueryCondition(hostnames);
+    stmt = PhoenixTransactSQL.prepareScanMetricMetadataSqlStmt(conn, metadataQueryCondition);
+    try {
+      if (stmt != null) {
+        rs = stmt.executeQuery();
+        while (rs.next()) {
+          byte[] uuid = rs.getBytes("UUID");
+          if (uuid != null) {
+            uuidList.add(uuid);
+          }
+        }
+      }
+    } finally {
+      if (rs != null) {
+        try {
+          rs.close();
+        } catch (SQLException e) {
+          // Ignore
+        }
+      }
+      if (stmt != null) {
+        try {
+          stmt.close();
+        } catch (SQLException e) {
+          // Ignore
+        }
+      }
+      if (conn != null) {
+        try {
+          conn.close();
+        } catch (SQLException sql) {
+          // Ignore
+        }
+      }
+    }
+    return uuidList;
+  }
+
   // No filter criteria support for now.
   public Map<TimelineMetricMetadataKey, TimelineMetricMetadata> getTimelineMetricMetadataV1() throws SQLException {
     Map<TimelineMetricMetadataKey, TimelineMetricMetadata> metadataMap = new HashMap<>();
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
index 2dedd77..7c6fce5 100644
--- a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/discovery/TimelineMetricMetadataManager.java
@@ -39,11 +39,11 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import org.apache.ambari.metrics.core.timeline.MetricsSystemInitializationException;
 import org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration;
 import org.apache.ambari.metrics.core.timeline.uuid.MetricUuidGenStrategy;
-import org.apache.ambari.metrics.core.timeline.uuid.MD5UuidGenStrategy;
 import org.apache.ambari.metrics.core.timeline.uuid.Murmur3HashUuidGenStrategy;
 import org.apache.ambari.metrics.core.timeline.uuid.TimelineMetricUuid;
 import org.apache.commons.collections.CollectionUtils;
@@ -64,7 +64,6 @@
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TRANSIENT_METRIC_PATTERNS;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.METRICS_METADATA_SYNC_INIT_DELAY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.METRICS_METADATA_SYNC_SCHEDULE_DELAY;
-import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRICS_UUID_GEN_STRATEGY;
 import static org.apache.ambari.metrics.core.timeline.TimelineMetricConfiguration.TIMELINE_METRIC_METADATA_FILTERS;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_HOSTED_APPS_METADATA_TABLE_SQL;
 import static org.apache.ambari.metrics.core.timeline.query.PhoenixTransactSQL.CREATE_INSTANCE_HOST_TABLE_SQL;
@@ -361,7 +360,8 @@
   }
 
   boolean isDistributedModeEnabled() {
-    return metricsConf.get("timeline.metrics.service.operation.mode").equals("distributed");
+    String mode = metricsConf.get("timeline.metrics.service.operation.mode");
+    return (mode != null) && mode.equals("distributed");
   }
 
   /**
@@ -629,75 +629,99 @@
                                                 String instanceId,
                                                 List<String> transientMetricNames) {
 
-    Collection<String> sanitizedMetricNames = new HashSet<>();
     List<byte[]> uuids = new ArrayList<>();
 
+    boolean metricNameHasWildcard = false;
     for (String metricName : metricNames) {
-      if (metricName.contains("%")) {
-        String metricRegEx = getJavaRegexFromSqlRegex(metricName);
-        for (TimelineMetricMetadataKey key : METADATA_CACHE.keySet()) {
-          String metricNameFromMetadata = key.getMetricName();
-          if (metricNameFromMetadata.matches(metricRegEx)) {
-            sanitizedMetricNames.add(metricNameFromMetadata);
-          }
-        }
-      } else {
-        sanitizedMetricNames.add(metricName);
+      if (hasWildCard(metricName)) {
+        metricNameHasWildcard = true;
+        break;
       }
     }
 
-    if(sanitizedMetricNames.isEmpty()) {
-      return uuids;
+    boolean hostNameHasWildcard = false;
+    if (CollectionUtils.isNotEmpty(hostnames)) {
+      for (String hostname : hostnames) {
+        if (hasWildCard(hostname)) {
+          hostNameHasWildcard = true;
+          break;
+        }
+      }
     }
 
-    Set<String> sanitizedHostNames = getSanitizedHostnames(hostnames);
-
     if ( StringUtils.isNotEmpty(appId) && !(appId.equals("HOST") || appId.equals("FLUME_HANDLER"))) { //HACK.. Why??
       appId = appId.toLowerCase();
     }
-    if (CollectionUtils.isNotEmpty(sanitizedHostNames)) {
-      if (CollectionUtils.isNotEmpty(sanitizedMetricNames)) {
 
-        //Skip getting UUID if it is a transient metric.
-        //An attempt to get it will also be OK as we don't add null UUIDs.
-        for (String metricName : sanitizedMetricNames) {
-          if (isTransientMetric(metricName, appId)) {
-            transientMetricNames.add(metricName);
-            continue;
+    if (hasWildCard(instanceId) || hasWildCard(appId) || hostNameHasWildcard || metricNameHasWildcard) {
+      try {
+        List<TimelineMetricMetadata> metricMetadataFromStore = hBaseAccessor.scanMetricMetadataForWildCardRequest(metricNames,
+          appId, instanceId);
+        List<byte[]> hostUuidsFromStore = hBaseAccessor.scanHostMetadataForWildCardRequest(hostnames);
+
+        for (TimelineMetricMetadata matchedEntry : metricMetadataFromStore) {
+          if (matchedEntry.getUuid() != null) {
+            if (CollectionUtils.isNotEmpty(hostnames)) {
+              for (byte[] hostUuidEntry : hostUuidsFromStore) {
+                uuids.add(ArrayUtils.addAll(matchedEntry.getUuid(), hostUuidEntry));
+              }
+            } else {
+              uuids.add(matchedEntry.getUuid());
+            }
+          } else if (isTransientMetric(matchedEntry.getMetricName(), matchedEntry.getAppId())) {
+            transientMetricNames.add(matchedEntry.getMetricName());
           }
-          TimelineMetric metric = new TimelineMetric();
-          metric.setMetricName(metricName);
-          metric.setAppId(appId);
-          metric.setInstanceId(instanceId);
-          for (String hostname : sanitizedHostNames) {
-            metric.setHostName(hostname);
-            byte[] uuid = getUuid(metric, false);
+        }
+        return uuids;
+      } catch (SQLException e) {
+        LOG.error("Unable to query metadata table to check satisfying metric keys for wildcard request : " + e);
+        return uuids;
+      }
+    } else {
+
+      if (CollectionUtils.isNotEmpty(hostnames)) {
+        if (CollectionUtils.isNotEmpty(metricNames)) {
+          //Skip getting UUID if it is a transient metric.
+          //An attempt to get it will also be OK as we don't add null UUIDs.
+          for (String metricName : metricNames) {
+            if (isTransientMetric(metricName, appId)) {
+              transientMetricNames.add(metricName);
+              continue;
+            }
+            TimelineMetric metric = new TimelineMetric();
+            metric.setMetricName(metricName);
+            metric.setAppId(appId);
+            metric.setInstanceId(instanceId);
+            for (String hostname : hostnames) {
+              metric.setHostName(hostname);
+              byte[] uuid = getUuid(metric, false);
+              if (uuid != null) {
+                uuids.add(uuid);
+              }
+            }
+          }
+        } else {
+          for (String hostname : hostnames) {
+            byte[] uuid = getUuidForHostname(hostname, false);
             if (uuid != null) {
               uuids.add(uuid);
             }
           }
         }
       } else {
-        for (String hostname : sanitizedHostNames) {
-          byte[] uuid = getUuidForHostname(hostname, false);
+        for (String metricName : metricNames) {
+          //Skip getting UUID if it is a transient metric. An attempt to get it will also be OK as we don't add null UUIDs.
+          if (isTransientMetric(metricName, appId)) {
+            transientMetricNames.add(metricName);
+            continue;
+          }
+          TimelineClusterMetric metric = new TimelineClusterMetric(metricName, appId, instanceId, -1l);
+          byte[] uuid = getUuid(metric, false);
           if (uuid != null) {
             uuids.add(uuid);
           }
         }
       }
-    } else {
-      for (String metricName : sanitizedMetricNames) {
-        //Skip getting UUID if it is a transient metric. An attempt to get it will also be OK as we don't add null UUIDs.
-        if (isTransientMetric(metricName, appId)) {
-          transientMetricNames.add(metricName);
-          continue;
-        }
-        TimelineClusterMetric metric = new TimelineClusterMetric(metricName, appId, instanceId, -1l);
-        byte[] uuid = getUuid(metric, false);
-        if (uuid != null) {
-          uuids.add(uuid);
-        }
-      }
     }
 
     return uuids;
@@ -833,6 +857,9 @@
    * Run TimelineMetricMetadataSync once
    */
   public void forceMetricsMetadataSync() {
+    if (metricMetadataSync == null) {
+      metricMetadataSync = new TimelineMetricMetadataSync(this);
+    }
     metricMetadataSync.run();
   }
 
@@ -860,4 +887,9 @@
       }
     }
   }
+
+  private boolean hasWildCard(String key) {
+    return StringUtils.isNotEmpty(key) && key.contains("%");
+  }
+
 }
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
index 7f72187..46204fc 100644
--- a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/DefaultCondition.java
@@ -28,7 +28,6 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.metrics2.sink.timeline.Precision;
 import org.apache.ambari.metrics.core.timeline.PhoenixHBaseAccessor;
-import org.apache.ambari.metrics.core.timeline.discovery.TimelineMetricMetadataManager;
 
 public class DefaultCondition implements Condition {
   List<String> metricNames;
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/MetadataQueryCondition.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/MetadataQueryCondition.java
new file mode 100644
index 0000000..f6e8c6a
--- /dev/null
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/MetadataQueryCondition.java
@@ -0,0 +1,77 @@
+/**
+ * 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.ambari.metrics.core.timeline.query;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.StringUtils;
+
+public class MetadataQueryCondition extends TransientMetricCondition {
+
+  private boolean isMetricMetadataCondition = true;
+
+  public MetadataQueryCondition(List<String> metricNames, String appId, String instanceId) {
+    super(metricNames, Collections.EMPTY_LIST, appId, instanceId, null, null, null , null, true, null);
+    this.hostnames = Collections.EMPTY_LIST;
+  }
+
+  public MetadataQueryCondition(List<String> hostnames) {
+    super(Collections.EMPTY_LIST, hostnames, StringUtils.EMPTY, StringUtils.EMPTY, null, null, null , null, true, null);
+    isMetricMetadataCondition = false;
+  }
+
+  public StringBuilder getConditionClause() {
+    StringBuilder sb = new StringBuilder();
+
+    boolean appendConjunction = false;
+    if (CollectionUtils.isNotEmpty(metricNames)) {
+      appendConjunction = appendMetricNameClause(sb);
+    }
+
+    if (CollectionUtils.isNotEmpty(hostnames)) {
+      appendConjunction = appendHostnameClause(sb, appendConjunction);
+    }
+
+    String appId = getAppId();
+    if (StringUtils.isNotEmpty(appId)) {
+      if (appId.contains("%")) {
+        appendConjunction = append(sb, appendConjunction, appId, " APP_ID LIKE ?");
+      } else {
+        appendConjunction = append(sb, appendConjunction, appId, " APP_ID = ?");
+      }
+    }
+
+    String instanceId = getInstanceId();
+    if (StringUtils.isNotEmpty(instanceId)) {
+      if (instanceId.contains("%")) {
+        appendConjunction = append(sb, appendConjunction, instanceId, " INSTANCE_ID LIKE ?");
+      } else {
+        appendConjunction = append(sb, appendConjunction, instanceId, " INSTANCE_ID = ?");
+      }
+    }
+
+    return sb;
+  }
+
+  public boolean isMetricMetadataCondition() {
+    return isMetricMetadataCondition;
+  }
+}
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
index 6a1d03f..ee50051 100644
--- a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/PhoenixTransactSQL.java
@@ -377,6 +377,10 @@
   public static final String GET_INSTANCE_HOST_METADATA_SQL = "SELECT " +
     "INSTANCE_ID, HOSTNAME FROM INSTANCE_HOST_METADATA";
 
+  public static final String SCAN_METRIC_METADATA_SQL = "SELECT METRIC_NAME, APP_ID, INSTANCE_ID, UUID FROM %s";
+
+  public static final String SCAN_HOST_METADATA_SQL = "SELECT HOSTNAME, UUID FROM %s";
+
   /**
    * Aggregate host metrics using a GROUP BY clause to take advantage of
    * N - way parallel scan where N = number of regions.
@@ -462,6 +466,12 @@
   public static final String METRICS_CLUSTER_AGGREGATE_DAILY_TABLE_NAME =
     "METRIC_AGGREGATE_DAILY_UUID";
 
+  public static final String METRICS_METADATA_TABLE_NAME =
+    "METRICS_METADATA_UUID";
+
+  public static final String HOST_METADATA_TABLE_NAME =
+    "HOSTED_APPS_METADATA_UUID";
+
   public static final Pattern PHOENIX_TABLES_REGEX_PATTERN = Pattern.compile("METRIC_.*");
 
   public static final String[] PHOENIX_TABLES = {
@@ -980,6 +990,46 @@
 
   }
 
+  public static PreparedStatement prepareScanMetricMetadataSqlStmt(Connection connection, MetadataQueryCondition condition)
+    throws SQLException {
+
+    String stmtStr;
+    if (condition.isMetricMetadataCondition()) {
+      stmtStr = String.format(SCAN_METRIC_METADATA_SQL, METRICS_METADATA_TABLE_NAME);
+    } else {
+      stmtStr = String.format(SCAN_HOST_METADATA_SQL, HOST_METADATA_TABLE_NAME);
+    }
+
+    StringBuilder sb = new StringBuilder(stmtStr);
+
+    sb.append(" WHERE ");
+    sb.append(condition.getConditionClause());
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SQL: " + sb.toString() + ", condition: " + condition);
+    }
+
+    PreparedStatement stmt = null;
+    try {
+      stmt = connection.prepareStatement(sb.toString());
+      int pos = 1;
+      if (condition.isMetricMetadataCondition()) {
+        pos = addMetricNames(condition, pos, stmt);
+        pos = addAppId(condition, pos, stmt);
+        pos = addInstanceId(condition, pos, stmt);
+      } else {
+        pos = addHostNames(condition, pos, stmt);
+      }
+
+    } catch (SQLException e) {
+      if (stmt != null) {
+        stmt.close();
+      }
+      throw e;
+    }
+    return stmt;
+  }
+
   public static String getTargetTableUsingPrecision(Precision precision, boolean withHosts) {
 
     String inputTable = null;
diff --git a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java
index c26d18d..b0f526c 100644
--- a/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java
+++ b/ambari-metrics-timelineservice/src/main/java/org/apache/ambari/metrics/core/timeline/query/TransientMetricCondition.java
@@ -42,6 +42,16 @@
     }
   }
 
+  public TransientMetricCondition(List<String> metricNames, List<String> hostnames, String appId,
+                                  String instanceId, Long startTime, Long endTime, Precision precision,
+                                  Integer limit, boolean grouped, List<String> transientMetricNames) {
+    super(metricNames, hostnames, appId, instanceId, startTime, endTime, precision, limit, grouped);
+    this.transientMetricNames = transientMetricNames;
+    if (CollectionUtils.isEmpty(hostnames)) {
+      this.hostnames = Collections.singletonList("%");
+    }
+  }
+
   public StringBuilder getTransientConditionClause() {
     StringBuilder sb = new StringBuilder();
 
@@ -49,15 +59,26 @@
 
     appendConjunction = appendHostnameClause(sb, appendConjunction);
 
-    appendConjunction = append(sb, appendConjunction, getAppId(), " APP_ID = ?");
-    appendConjunction = append(sb, appendConjunction, getInstanceId(), " INSTANCE_ID = ?");
+    String appId = getAppId();
+    if (appId.contains("%")) {
+      appendConjunction = append(sb, appendConjunction, getAppId(), " APP_ID LIKE ?");
+    } else {
+      appendConjunction = append(sb, appendConjunction, getAppId(), " APP_ID = ?");
+    }
+
+    String instanceId = getInstanceId();
+    if (instanceId.contains("%")) {
+      appendConjunction = append(sb, appendConjunction, getInstanceId(), " INSTANCE_ID LIKE ?");
+    } else {
+      appendConjunction = append(sb, appendConjunction, getInstanceId(), " INSTANCE_ID = ?");
+    }
     appendConjunction = append(sb, appendConjunction, getStartTime(), " SERVER_TIME >= ?");
     append(sb, appendConjunction, getEndTime(), " SERVER_TIME < ?");
 
     return sb;
   }
 
-  private boolean appendMetricNameClause(StringBuilder sb) {
+  protected boolean appendMetricNameClause(StringBuilder sb) {
     boolean appendConjunction = false;
     List<String> metricsLike = new ArrayList<>();
     List<String> metricsIn = new ArrayList<>();
diff --git a/ambari-metrics-timelineservice/src/test/java/org/apache/ambari/metrics/core/timeline/discovery/TestMetadataManager.java b/ambari-metrics-timelineservice/src/test/java/org/apache/ambari/metrics/core/timeline/discovery/TestMetadataManager.java
index 90def86..90a5d10 100644
--- a/ambari-metrics-timelineservice/src/test/java/org/apache/ambari/metrics/core/timeline/discovery/TestMetadataManager.java
+++ b/ambari-metrics-timelineservice/src/test/java/org/apache/ambari/metrics/core/timeline/discovery/TestMetadataManager.java
@@ -181,7 +181,7 @@
   }
 
   @Test
-  public void testWildcardSanitization() throws IOException, SQLException, URISyntaxException {
+  public void testMetricHostWildcardSanitization() throws IOException, SQLException, URISyntaxException {
     // Initialize new manager
     metadataManager = new TimelineMetricMetadataManager(new Configuration(), hdb);
     hdb.setMetadataInstance(metadataManager);
@@ -234,6 +234,7 @@
     replay(configuration);
 
     hdb.insertMetricRecordsWithMetadata(metadataManager, timelineMetrics, true);
+    metadataManager.forceMetricsMetadataSync();
 
     List<byte[]> uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("dummy_m%"),
       Collections.singletonList("dummy_host2"), "dummy_app1", null);
@@ -255,6 +256,85 @@
   }
 
   @Test
+  public void testAppInstanceWildcardSanitization() throws IOException, SQLException, URISyntaxException {
+    // Initialize new manager
+    metadataManager = new TimelineMetricMetadataManager(new Configuration(), hdb);
+    hdb.setMetadataInstance(metadataManager);
+    final long now = System.currentTimeMillis();
+
+    TimelineMetrics timelineMetrics = new TimelineMetrics();
+
+    TimelineMetric metric1 = new TimelineMetric();
+    metric1.setMetricName("dummy_m1");
+    metric1.setHostName("dummy_host1");
+    metric1.setStartTime(now - 1000);
+    metric1.setAppId("dummy_app1");
+    metric1.setInstanceId("dummy_i1");
+    metric1.setType("Integer");
+    metric1.setMetricValues(new TreeMap<Long, Double>() {{
+      put(now - 100, 1.0);
+      put(now - 200, 2.0);
+      put(now - 300, 3.0);
+    }});
+    timelineMetrics.getMetrics().add(metric1);
+
+    TimelineMetric metric2 = new TimelineMetric();
+    metric2.setMetricName("dummy_m2");
+    metric2.setHostName("dummy_host2");
+    metric2.setStartTime(now - 1000);
+    metric2.setAppId("dummy_app2");
+    metric2.setInstanceId("dummy_i2");
+    metric2.setType("Integer");
+    metric2.setMetricValues(new TreeMap<Long, Double>() {{
+      put(now - 100, 1.0);
+      put(now - 200, 2.0);
+      put(now - 300, 3.0);
+    }});
+    timelineMetrics.getMetrics().add(metric2);
+
+    TimelineMetric metric3 = new TimelineMetric();
+    metric3.setMetricName("dummy_3");
+    metric3.setHostName("gummy_3h");
+    metric3.setStartTime(now - 1000);
+    metric3.setAppId("gummy_app3");
+    metric3.setType("Integer");
+    metric3.setMetricValues(new TreeMap<Long, Double>() {{
+      put(now - 100, 1.0);
+      put(now - 200, 2.0);
+      put(now - 300, 3.0);
+    }});
+    timelineMetrics.getMetrics().add(metric3);
+
+    Configuration metricsConf = new Configuration();
+    TimelineMetricConfiguration configuration = EasyMock.createNiceMock(TimelineMetricConfiguration.class);
+    expect(configuration.getMetricsConf()).andReturn(metricsConf).once();
+    replay(configuration);
+
+    hdb.insertMetricRecordsWithMetadata(metadataManager, timelineMetrics, true);
+    metadataManager.forceMetricsMetadataSync();
+
+    List<byte[]> uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("dummy_m%"),
+      Collections.singletonList("dummy_host%"), "dummy_app%", "dummy_i%");
+    Assert.assertTrue(uuids.size() == 4);
+
+    uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("dummy_m1"),
+      Collections.singletonList("dummy_host1"), "dummy_app1", "%");
+    Assert.assertTrue(uuids.size() == 1);
+
+    uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("dummy_m1"),
+      Collections.singletonList("dummy_host1"), "dummy_app1", "dummy_i%");
+    Assert.assertTrue(uuids.size() == 1);
+
+    uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("dummy_m1"),
+      Collections.singletonList("dummy_host1"), "dummy_app1", "dummy_i2");
+    Assert.assertTrue(uuids.size() == 0);
+
+    uuids = metadataManager.getUuidsForGetMetricQuery(Collections.singletonList("%"),
+      Collections.singletonList("%"), "%", "%");
+    Assert.assertTrue(uuids.size() == 6);
+  }
+
+  @Test
   public void testTransientMetricPatterns() {
 
     long now = System.currentTimeMillis();