Add EventMetricProducer

+ Started to use ProtoOutputStream in EventMetricProducer.
  [TODO]: We need to auto-generate fieldIds for StatsLogReport, XXXMetricData, etc.
  [TODO]: We need to add Enum type to liblog, otherwise we cannot reconstruct a proto containing
          an enum

+ Some refactor in metric initialization code. There are still boiler plate code, because Metrics
  are similar but with subtle differences.

Test: statsd_test

Change-Id: Id7e3212566249a8139b9680f04238c455d50c1b8
diff --git a/cmds/statsd/Android.mk b/cmds/statsd/Android.mk
index 8946aed..dd9a965 100644
--- a/cmds/statsd/Android.mk
+++ b/cmds/statsd/Android.mk
@@ -38,6 +38,7 @@
     src/matchers/SimpleLogMatchingTracker.cpp \
     src/metrics/CountAnomalyTracker.cpp \
     src/metrics/MetricProducer.cpp \
+    src/metrics/EventMetricProducer.cpp \
     src/metrics/CountMetricProducer.cpp \
     src/metrics/DurationMetricProducer.cpp \
     src/metrics/MetricsManager.cpp \
@@ -65,7 +66,8 @@
     libselinux \
     libutils \
     libservices \
-    libandroidfw
+    libandroidfw \
+    libprotoutil
 
 # =========
 # statsd
diff --git a/cmds/statsd/src/condition/SimpleConditionTracker.cpp b/cmds/statsd/src/condition/SimpleConditionTracker.cpp
index bde3846..aff4768 100644
--- a/cmds/statsd/src/condition/SimpleConditionTracker.cpp
+++ b/cmds/statsd/src/condition/SimpleConditionTracker.cpp
@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-#define DEBUG true  // STOPSHIP if true
+#define DEBUG false  // STOPSHIP if true
 #include "Log.h"
 
 #include "SimpleConditionTracker.h"
diff --git a/cmds/statsd/src/config/ConfigManager.cpp b/cmds/statsd/src/config/ConfigManager.cpp
index 88c7ebf..83ddffc 100644
--- a/cmds/statsd/src/config/ConfigManager.cpp
+++ b/cmds/statsd/src/config/ConfigManager.cpp
@@ -120,7 +120,7 @@
 
     int WAKE_LOCK_TAG_ID = 11;
     int WAKE_LOCK_UID_KEY_ID = 1;
-    int WAKE_LOCK_STATE_KEY = 2;
+    int WAKE_LOCK_STATE_KEY = 3;
     int WAKE_LOCK_ACQUIRE_VALUE = 1;
     int WAKE_LOCK_RELEASE_VALUE = 0;
 
@@ -161,7 +161,7 @@
     keyMatcher->set_key(UID_PROCESS_STATE_UID_KEY);
     metric->set_condition("SCREEN_IS_OFF");
 
-    // Count wake lock, slice by uid, while SCREEN_IS_OFF and app in background
+    // Count wake lock, slice by uid, while SCREEN_IS_ON and app in background
     metric = config.add_count_metric();
     metric->set_metric_id(4);
     metric->set_what("APP_GET_WL");
@@ -189,6 +189,11 @@
     link->add_key_in_main()->set_key(WAKE_LOCK_UID_KEY_ID);
     link->add_key_in_condition()->set_key(APP_USAGE_UID_KEY_ID);
 
+    // Add an EventMetric to log process state change events.
+    EventMetric* eventMetric = config.add_event_metric();
+    eventMetric->set_metric_id(6);
+    eventMetric->set_what("SCREEN_TURNED_ON");
+
     // Event matchers............
     LogEntryMatcher* eventMatcher = config.add_log_entry_matcher();
     eventMatcher->set_name("SCREEN_TURNED_ON");
diff --git a/cmds/statsd/src/logd/LogEvent.cpp b/cmds/statsd/src/logd/LogEvent.cpp
index fb992c1..1a039f6 100644
--- a/cmds/statsd/src/logd/LogEvent.cpp
+++ b/cmds/statsd/src/logd/LogEvent.cpp
@@ -17,6 +17,7 @@
 #include "logd/LogEvent.h"
 
 #include <sstream>
+#include "stats_util.h"
 
 namespace android {
 namespace os {
@@ -24,6 +25,7 @@
 
 using std::ostringstream;
 using std::string;
+using android::util::ProtoOutputStream;
 
 // We need to keep a copy of the android_log_event_list owned by this instance so that the char*
 // for strings is not cleared before we can read them.
@@ -203,30 +205,24 @@
     return result.str();
 }
 
-void LogEvent::ToProto(EventMetricData* out) const {
-    // TODO: Implement this when we have the ProtoOutputStream version.
-
-    // set timestamp of the event.
-    out->set_timestamp_nanos(mTimestampNs);
-
-    // uint64_t token = proto->StartObject(EventMetricData.FIELD);
+void LogEvent::ToProto(ProtoOutputStream& proto) const {
+    long long atomToken = proto.start(TYPE_MESSAGE + mTagId);
     const size_t N = mElements.size();
     for (size_t i=0; i<N; i++) {
         const int key = i + 1;
 
         const android_log_list_element& elem = mElements[i];
         if (elem.type == EVENT_TYPE_INT) {
-            // proto->Write(key, elem.data.int32);
+            proto.write(TYPE_INT32 + key, elem.data.int32);
         } else if (elem.type == EVENT_TYPE_LONG) {
-            // proto->Write(key, elem.data.int64);
+            proto.write(TYPE_INT64 + key, (long long)elem.data.int64);
         } else if (elem.type == EVENT_TYPE_FLOAT) {
-            // proto->Write(key, elem.data.float32);
+            proto.write(TYPE_FLOAT + key, elem.data.float32);
         } else if (elem.type == EVENT_TYPE_STRING) {
-            // proto->Write(key, elem.data.string);
+            proto.write(TYPE_STRING + key, elem.data.string);
         }
     }
-
-    //proto->EndObject(token);
+    proto.end(atomToken);
 }
 
 }  // namespace statsd
diff --git a/cmds/statsd/src/logd/LogEvent.h b/cmds/statsd/src/logd/LogEvent.h
index 4102675..9ef20ea 100644
--- a/cmds/statsd/src/logd/LogEvent.h
+++ b/cmds/statsd/src/logd/LogEvent.h
@@ -18,9 +18,10 @@
 
 #include "frameworks/base/cmds/statsd/src/stats_log.pb.h"
 
-#include <utils/Errors.h>
+#include <android/util/ProtoOutputStream.h>
 #include <log/log_event_list.h>
 #include <log/log_read.h>
+#include <utils/Errors.h>
 
 #include <memory>
 #include <string>
@@ -80,10 +81,9 @@
     string ToString() const;
 
     /**
-     * Write this object as an EventMetricData proto object.
-     * TODO: Use the streaming output generator to do this instead of this proto lite object?
+     * Write this object to a ProtoOutputStream.
      */
-    void ToProto(EventMetricData* out) const;
+    void ToProto(android::util::ProtoOutputStream& out) const;
 
     /*
      * Get a KeyValuePair proto object.
diff --git a/cmds/statsd/src/metrics/EventMetricProducer.cpp b/cmds/statsd/src/metrics/EventMetricProducer.cpp
new file mode 100644
index 0000000..8b3f405
--- /dev/null
+++ b/cmds/statsd/src/metrics/EventMetricProducer.cpp
@@ -0,0 +1,132 @@
+/*
+ * Copyright (C) 2017 The Android Open Source Project
+ *
+ * Licensed 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.
+ */
+
+#define DEBUG true  // STOPSHIP if true
+#include "Log.h"
+
+#include "EventMetricProducer.h"
+#include "stats_util.h"
+
+#include <cutils/log.h>
+#include <limits.h>
+#include <stdlib.h>
+
+using android::util::ProtoOutputStream;
+using std::map;
+using std::string;
+using std::unordered_map;
+using std::vector;
+
+namespace android {
+namespace os {
+namespace statsd {
+
+// for StatsLogReport
+const int FIELD_ID_METRIC_ID = 1;
+const int FIELD_ID_START_REPORT_NANOS = 2;
+const int FIELD_ID_END_REPORT_NANOS = 2;
+const int FIELD_ID_EVENT_METRICS = 4;
+// for EventMetricData
+const int FIELD_ID_TIMESTAMP_NANOS = 1;
+const int FIELD_ID_STATS_EVENTS = 2;
+// for CountMetricDataWrapper
+const int FIELD_ID_DATA = 1;
+
+EventMetricProducer::EventMetricProducer(const EventMetric& metric, const int conditionIndex,
+                                         const sp<ConditionWizard>& wizard)
+    // TODO: Pass in the start time from MetricsManager, instead of calling time() here.
+    : MetricProducer((time(nullptr) * NANO_SECONDS_IN_A_SECOND), conditionIndex, wizard),
+      mMetric(metric) {
+    if (metric.links().size() > 0) {
+        mConditionLinks.insert(mConditionLinks.begin(), metric.links().begin(),
+                               metric.links().end());
+        mConditionSliced = true;
+    }
+
+    startNewProtoOutputStream(mStartTimeNs);
+
+    VLOG("metric %lld created. bucket size %lld start_time: %lld", metric.metric_id(),
+         (long long)mBucketSizeNs, (long long)mStartTimeNs);
+}
+
+EventMetricProducer::~EventMetricProducer() {
+    VLOG("~EventMetricProducer() called");
+}
+
+void EventMetricProducer::startNewProtoOutputStream(long long startTime) {
+    mProto = std::make_unique<ProtoOutputStream>();
+    // TODO: We need to auto-generate the field IDs for StatsLogReport, EventMetricData,
+    // and StatsEvent.
+    mProto->write(TYPE_INT32 + FIELD_ID_METRIC_ID, mMetric.metric_id());
+    mProto->write(TYPE_INT64 + FIELD_ID_START_REPORT_NANOS, startTime);
+    mProtoToken = mProto->start(TYPE_MESSAGE + FIELD_ID_EVENT_METRICS);
+}
+
+void EventMetricProducer::finish() {
+}
+
+void EventMetricProducer::onSlicedConditionMayChange() {
+}
+
+StatsLogReport EventMetricProducer::onDumpReport() {
+    long long endTime = time(nullptr) * NANO_SECONDS_IN_A_SECOND;
+    mProto->end(mProtoToken);
+    mProto->write(TYPE_INT64 + FIELD_ID_END_REPORT_NANOS, endTime);
+
+    size_t bufferSize = mProto->size();
+    VLOG("metric %lld dump report now... proto size: %zu ", mMetric.metric_id(), bufferSize);
+    std::unique_ptr<uint8_t[]> buffer(new uint8_t[bufferSize]);
+    size_t pos = 0;
+    auto it = mProto->data();
+    while (it.readBuffer() != NULL) {
+        size_t toRead = it.currentToRead();
+        std::memcpy(&buffer[pos], it.readBuffer(), toRead);
+        pos += toRead;
+        it.rp()->move(toRead);
+    }
+
+    startNewProtoOutputStream(endTime);
+
+    // TODO: Once we migrate all MetricProducers to use ProtoOutputStream, we should return this:
+    // return std::move(buffer);
+    return StatsLogReport();
+}
+
+void EventMetricProducer::onConditionChanged(const bool conditionMet) {
+    VLOG("Metric %lld onConditionChanged", mMetric.metric_id());
+    mCondition = conditionMet;
+}
+
+void EventMetricProducer::onMatchedLogEventInternal(
+        const size_t matcherIndex, const HashableDimensionKey& eventKey,
+        const std::map<std::string, HashableDimensionKey>& conditionKey, bool condition,
+        const LogEvent& event) {
+
+    if (!condition) {
+        return;
+    }
+
+    long long wrapperToken = mProto->start(TYPE_MESSAGE + FIELD_ID_DATA);
+    mProto->write(TYPE_INT64 + FIELD_ID_TIMESTAMP_NANOS, (long long)event.GetTimestampNs());
+    long long eventToken = mProto->start(TYPE_MESSAGE + FIELD_ID_STATS_EVENTS);
+    event.ToProto(*mProto);
+    mProto->end(eventToken);
+    mProto->end(wrapperToken);
+}
+
+}  // namespace statsd
+}  // namespace os
+}  // namespace android
diff --git a/cmds/statsd/src/metrics/EventMetricProducer.h b/cmds/statsd/src/metrics/EventMetricProducer.h
new file mode 100644
index 0000000..879175c
--- /dev/null
+++ b/cmds/statsd/src/metrics/EventMetricProducer.h
@@ -0,0 +1,70 @@
+/*
+ * Copyright (C) 2017 The Android Open Source Project
+ *
+ * Licensed 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.
+ */
+
+#ifndef EVENT_METRIC_PRODUCER_H
+#define EVENT_METRIC_PRODUCER_H
+
+#include <unordered_map>
+
+#include <android/util/ProtoOutputStream.h>
+#include "../condition/ConditionTracker.h"
+#include "../matchers/matcher_util.h"
+#include "MetricProducer.h"
+#include "frameworks/base/cmds/statsd/src/stats_log.pb.h"
+#include "frameworks/base/cmds/statsd/src/statsd_config.pb.h"
+#include "stats_util.h"
+
+namespace android {
+namespace os {
+namespace statsd {
+
+class EventMetricProducer : public MetricProducer {
+public:
+    // TODO: Pass in the start time from MetricsManager, it should be consistent for all metrics.
+    EventMetricProducer(const EventMetric& eventMetric, const int conditionIndex,
+                        const sp<ConditionWizard>& wizard);
+
+    virtual ~EventMetricProducer();
+
+    void onMatchedLogEventInternal(const size_t matcherIndex, const HashableDimensionKey& eventKey,
+                                   const std::map<std::string, HashableDimensionKey>& conditionKey,
+                                   bool condition, const LogEvent& event) override;
+
+    void onConditionChanged(const bool conditionMet) override;
+
+    void finish() override;
+
+    StatsLogReport onDumpReport() override;
+
+    void onSlicedConditionMayChange() override;
+
+    // TODO: Implement this later.
+    virtual void notifyAppUpgrade(const string& apk, const int uid, const int version) override{};
+
+private:
+    const EventMetric mMetric;
+
+    std::unique_ptr<android::util::ProtoOutputStream> mProto;
+
+    long long mProtoToken;
+
+    void startNewProtoOutputStream(long long timestamp);
+};
+
+}  // namespace statsd
+}  // namespace os
+}  // namespace android
+#endif  // EVENT_METRIC_PRODUCER_H
diff --git a/cmds/statsd/src/metrics/metrics_manager_util.cpp b/cmds/statsd/src/metrics/metrics_manager_util.cpp
index 23071aa..e90f998 100644
--- a/cmds/statsd/src/metrics/metrics_manager_util.cpp
+++ b/cmds/statsd/src/metrics/metrics_manager_util.cpp
@@ -20,6 +20,7 @@
 #include "../matchers/SimpleLogMatchingTracker.h"
 #include "CountMetricProducer.h"
 #include "DurationMetricProducer.h"
+#include "EventMetricProducer.h"
 #include "stats_util.h"
 
 using std::set;
@@ -31,13 +32,51 @@
 namespace os {
 namespace statsd {
 
-int getTrackerIndex(const string& name, const unordered_map<string, int>& logTrackerMap) {
-    auto logTrackerIt = logTrackerMap.find(name);
+bool handleMetricWithLogTrackers(const string what, const int metricIndex,
+                                 const unordered_map<string, int>& logTrackerMap,
+                                 unordered_map<int, std::vector<int>>& trackerToMetricMap,
+                                 int& logTrackerIndex) {
+    auto logTrackerIt = logTrackerMap.find(what);
     if (logTrackerIt == logTrackerMap.end()) {
-        ALOGW("cannot find the LogEventMatcher %s in config", name.c_str());
-        return MATCHER_NOT_FOUND;
+        ALOGW("cannot find the LogEntryMatcher %s in config", what.c_str());
+        return false;
     }
-    return logTrackerIt->second;
+    logTrackerIndex = logTrackerIt->second;
+    auto& metric_list = trackerToMetricMap[logTrackerIndex];
+    metric_list.push_back(metricIndex);
+    return true;
+}
+
+bool handleMetricWithConditions(
+        const string condition, const int metricIndex,
+        const unordered_map<string, int>& conditionTrackerMap,
+        const ::google::protobuf::RepeatedPtrField<::android::os::statsd::EventConditionLink>&
+                links,
+        vector<sp<ConditionTracker>>& allConditionTrackers, int& conditionIndex,
+        unordered_map<int, std::vector<int>>& conditionToMetricMap) {
+    auto condition_it = conditionTrackerMap.find(condition);
+    if (condition_it == conditionTrackerMap.end()) {
+        ALOGW("cannot find the Condition %s in the config", condition.c_str());
+        return false;
+    }
+
+    for (const auto& link : links) {
+        auto it = conditionTrackerMap.find(link.condition());
+        if (it == conditionTrackerMap.end()) {
+            ALOGW("cannot find the Condition %s in the config", link.condition().c_str());
+            return false;
+        }
+        allConditionTrackers[condition_it->second]->setSliced(true);
+        allConditionTrackers[it->second]->setSliced(true);
+        allConditionTrackers[it->second]->addDimensions(
+                vector<KeyMatcher>(link.key_in_condition().begin(), link.key_in_condition().end()));
+    }
+    conditionIndex = condition_it->second;
+
+    // will create new vector if not exist before.
+    auto& metricList = conditionToMetricMap[condition_it->second];
+    metricList.push_back(metricIndex);
+    return true;
 }
 
 bool initLogTrackers(const StatsdConfig& config, unordered_map<string, int>& logTrackerMap,
@@ -142,7 +181,8 @@
                  unordered_map<int, std::vector<int>>& conditionToMetricMap,
                  unordered_map<int, std::vector<int>>& trackerToMetricMap) {
     sp<ConditionWizard> wizard = new ConditionWizard(allConditionTrackers);
-    const int allMetricsCount = config.count_metric_size() + config.duration_metric_size();
+    const int allMetricsCount =
+            config.count_metric_size() + config.duration_metric_size() + config.event_metric_size();
     allMetricProducers.reserve(allMetricsCount);
 
     // Build MetricProducers for each metric defined in config.
@@ -155,100 +195,52 @@
         }
 
         int metricIndex = allMetricProducers.size();
-
-        auto logTrackerIt = logTrackerMap.find(metric.what());
-        if (logTrackerIt == logTrackerMap.end()) {
-            ALOGW("cannot find the LogEntryMatcher %s in config", metric.what().c_str());
+        int trackerIndex;
+        if (!handleMetricWithLogTrackers(metric.what(), metricIndex, logTrackerMap,
+                                         trackerToMetricMap, trackerIndex)) {
             return false;
         }
-        int logTrackerIndex = logTrackerIt->second;
-        auto& metric_list = trackerToMetricMap[logTrackerIndex];
-        metric_list.push_back(metricIndex);
 
-        sp<MetricProducer> countProducer;
-
+        int conditionIndex = -1;
         if (metric.has_condition()) {
-            auto condition_it = conditionTrackerMap.find(metric.condition());
-            if (condition_it == conditionTrackerMap.end()) {
-                ALOGW("cannot find the Condition %s in the config", metric.condition().c_str());
-                return false;
-            }
-
-            for (const auto& link : metric.links()) {
-                auto it = conditionTrackerMap.find(link.condition());
-                if (it == conditionTrackerMap.end()) {
-                    ALOGW("cannot find the Condition %s in the config", link.condition().c_str());
-                    return false;
-                }
-                allConditionTrackers[condition_it->second]->setSliced(true);
-                allConditionTrackers[it->second]->setSliced(true);
-                allConditionTrackers[it->second]->addDimensions(vector<KeyMatcher>(
-                        link.key_in_condition().begin(), link.key_in_condition().end()));
-            }
-
-            countProducer = new CountMetricProducer(metric, condition_it->second, wizard);
-            // will create new vector if not exist before.
-            auto& metricList = conditionToMetricMap[condition_it->second];
-            metricList.push_back(metricIndex);
-        } else {
-            countProducer = new CountMetricProducer(metric, -1 /*no condition*/, wizard);
+            handleMetricWithConditions(metric.condition(), metricIndex, conditionTrackerMap,
+                                       metric.links(), allConditionTrackers, conditionIndex,
+                                       conditionToMetricMap);
         }
+
+        sp<MetricProducer> countProducer = new CountMetricProducer(metric, conditionIndex, wizard);
         allMetricProducers.push_back(countProducer);
     }
 
     for (int i = 0; i < config.duration_metric_size(); i++) {
         int metricIndex = allMetricProducers.size();
-        const DurationMetric metric = config.duration_metric(i);
-        if (!metric.has_start()) {
-            ALOGW("cannot find start in DurationMetric %lld", metric.metric_id());
+        const DurationMetric& metric = config.duration_metric(i);
+        int trackerIndices[3] = {-1, -1, -1};
+        if (!metric.has_start() ||
+            !handleMetricWithLogTrackers(metric.start(), metricIndex, logTrackerMap,
+                                         trackerToMetricMap, trackerIndices[0])) {
+            ALOGE("Duration metrics must specify a valid the start event matcher");
             return false;
         }
 
-        int trackerIndices[] = {-1, -1, -1};
-        trackerIndices[0] = getTrackerIndex(metric.start(), logTrackerMap);
-
-        if (metric.has_stop()) {
-            trackerIndices[1] = getTrackerIndex(metric.stop(), logTrackerMap);
+        if (metric.has_stop() &&
+            !handleMetricWithLogTrackers(metric.stop(), metricIndex, logTrackerMap,
+                                         trackerToMetricMap, trackerIndices[1])) {
+            return false;
         }
 
-        if (metric.has_stop_all()) {
-            trackerIndices[2] = getTrackerIndex(metric.stop_all(), logTrackerMap);
-        }
-
-        for (const int& index : trackerIndices) {
-            if (index == MATCHER_NOT_FOUND) {
-                return false;
-            }
-            if (index >= 0) {
-                auto& metric_list = trackerToMetricMap[index];
-                metric_list.push_back(metricIndex);
-            }
+        if (metric.has_stop_all() &&
+            !handleMetricWithLogTrackers(metric.stop_all(), metricIndex, logTrackerMap,
+                                         trackerToMetricMap, trackerIndices[2])) {
+            return false;
         }
 
         int conditionIndex = -1;
 
         if (metric.has_predicate()) {
-            auto condition_it = conditionTrackerMap.find(metric.predicate());
-            if (condition_it == conditionTrackerMap.end()) {
-                ALOGW("cannot find the Condition %s in the config", metric.predicate().c_str());
-                return false;
-            }
-            conditionIndex = condition_it->second;
-
-            for (const auto& link : metric.links()) {
-                auto it = conditionTrackerMap.find(link.condition());
-                if (it == conditionTrackerMap.end()) {
-                    ALOGW("cannot find the Condition %s in the config", link.condition().c_str());
-                    return false;
-                }
-                allConditionTrackers[condition_it->second]->setSliced(true);
-                allConditionTrackers[it->second]->setSliced(true);
-                allConditionTrackers[it->second]->addDimensions(vector<KeyMatcher>(
-                        link.key_in_condition().begin(), link.key_in_condition().end()));
-            }
-
-            auto& metricList = conditionToMetricMap[conditionIndex];
-            metricList.push_back(metricIndex);
+            handleMetricWithConditions(metric.predicate(), metricIndex, conditionTrackerMap,
+                                       metric.links(), allConditionTrackers, conditionIndex,
+                                       conditionToMetricMap);
         }
 
         sp<MetricProducer> durationMetric =
@@ -257,6 +249,32 @@
 
         allMetricProducers.push_back(durationMetric);
     }
+
+    for (int i = 0; i < config.event_metric_size(); i++) {
+        int metricIndex = allMetricProducers.size();
+        const EventMetric& metric = config.event_metric(i);
+        if (!metric.has_metric_id() || !metric.has_what()) {
+            ALOGW("cannot find the metric id or what in config");
+            return false;
+        }
+        int trackerIndex;
+        if (!handleMetricWithLogTrackers(metric.what(), metricIndex, logTrackerMap,
+                                         trackerToMetricMap, trackerIndex)) {
+            return false;
+        }
+
+        int conditionIndex = -1;
+        if (metric.has_condition()) {
+            handleMetricWithConditions(metric.condition(), metricIndex, conditionTrackerMap,
+                                       metric.links(), allConditionTrackers, conditionIndex,
+                                       conditionToMetricMap);
+        }
+
+        sp<MetricProducer> eventMetric = new EventMetricProducer(metric, conditionIndex, wizard);
+
+        allMetricProducers.push_back(eventMetric);
+    }
+
     return true;
 }
 
diff --git a/cmds/statsd/src/metrics/metrics_manager_util.h b/cmds/statsd/src/metrics/metrics_manager_util.h
index 38149a6..88923fe 100644
--- a/cmds/statsd/src/metrics/metrics_manager_util.h
+++ b/cmds/statsd/src/metrics/metrics_manager_util.h
@@ -22,7 +22,6 @@
 
 #include "../condition/ConditionTracker.h"
 #include "../matchers/LogMatchingTracker.h"
-#include "CountMetricProducer.h"
 
 namespace android {
 namespace os {
@@ -91,8 +90,6 @@
                       std::unordered_map<int, std::vector<int>>& trackerToMetricMap,
                       std::unordered_map<int, std::vector<int>>& trackerToConditionMap);
 
-int getTrackerIndex(const std::string& name, const std::unordered_map<string, int>& logTrackerMap);
-
 }  // namespace statsd
 }  // namespace os
 }  // namespace android
diff --git a/cmds/statsd/src/stats_util.h b/cmds/statsd/src/stats_util.h
index 575588b..39c1d59 100644
--- a/cmds/statsd/src/stats_util.h
+++ b/cmds/statsd/src/stats_util.h
@@ -30,6 +30,14 @@
 #define MATCHER_NOT_FOUND -2
 #define NANO_SECONDS_IN_A_SECOND (1000 * 1000 * 1000)
 
+// TODO: Remove the following constants once they are exposed in ProtOutputStream.h
+const uint64_t FIELD_TYPE_SHIFT = 32;
+const uint64_t TYPE_MESSAGE = 11ULL << FIELD_TYPE_SHIFT;
+const uint64_t TYPE_INT64 = 3ULL << FIELD_TYPE_SHIFT;
+const uint64_t TYPE_INT32 = 5ULL << FIELD_TYPE_SHIFT;
+const uint64_t TYPE_FLOAT = 2ULL << FIELD_TYPE_SHIFT;
+const uint64_t TYPE_STRING = 9ULL << FIELD_TYPE_SHIFT;
+
 typedef std::string HashableDimensionKey;
 
 EventMetricData parse(log_msg msg);