From 21069c0be96c034ded637706509338819965598b Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 2 Sep 2021 17:01:48 +0100 Subject: [PATCH] [Issue #11799][logging] Base implementation of structured logging (#11901) Includes the most simple parts of slf4j structured logging. timed(), sampled() and enum logging to follow. --- buildtools/src/main/resources/log4j2.xml | 2 +- pom.xml | 9 + structured-event-log/pom.xml | 5 + .../EventResourcesImpl.java | 85 +++++ .../StructuredEventLog.java | 4 +- .../structuredeventlog/slf4j/Slf4jEvent.java | 206 ++++++++++ .../slf4j/Slf4jStructuredEventLog.java | 43 +++ .../StructuredEventLogTest.java | 359 ++++++++++++++++++ 8 files changed, 711 insertions(+), 2 deletions(-) create mode 100644 structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/EventResourcesImpl.java create mode 100644 structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jEvent.java create mode 100644 structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jStructuredEventLog.java create mode 100644 structured-event-log/src/test/java/org/apache/pulsar/structuredeventlog/StructuredEventLogTest.java diff --git a/buildtools/src/main/resources/log4j2.xml b/buildtools/src/main/resources/log4j2.xml index db0935e30257b..a43c7e813484b 100644 --- a/buildtools/src/main/resources/log4j2.xml +++ b/buildtools/src/main/resources/log4j2.xml @@ -22,7 +22,7 @@ - + diff --git a/pom.xml b/pom.xml index 9699e9b179fa6..e74d7867d5f89 100644 --- a/pom.xml +++ b/pom.xml @@ -203,6 +203,8 @@ flexible messaging model and an intuitive client API. 3.6.0 3.4.0 1.15.3 + 2.2 + 3.2.8 1.1.1 @@ -277,6 +279,13 @@ flexible messaging model and an intuitive client API. + + org.hamcrest + hamcrest + ${hamcrest.version} + test + + junit junit diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index de872f8721d9a..cb882523fbafe 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -42,6 +42,11 @@ testng test + + org.hamcrest + hamcrest + test + org.apache.logging.log4j log4j-slf4j-impl diff --git a/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/EventResourcesImpl.java b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/EventResourcesImpl.java new file mode 100644 index 0000000000000..80df37e96ed34 --- /dev/null +++ b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/EventResourcesImpl.java @@ -0,0 +1,85 @@ +/** + * 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.pulsar.structuredeventlog; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.Supplier; + +public class EventResourcesImpl implements EventResources { + private final EventResourcesImpl parent; + private List resources = null; + + public EventResourcesImpl(EventResourcesImpl parent) { + this.parent = parent; + } + + @Override + public EventResources resource(String key, Object value) { + getResources().add(key); + getResources().add(value); + return this; + } + + @Override + public EventResources resource(String key, Supplier value) { + resource(key, (Object)value); + return this; + } + + public void copyFrom(EventResourcesImpl other) { + // can't use forEach because we want to avoid toString at this point + List resources = getResources(); + if (other.parent != null) { + copyFrom(other.parent); + } + + if (other.resources != null) { + resources.addAll(other.resources); + } + } + + public void forEach(BiConsumer process) { + if (parent != null) { + parent.forEach(process); + } + if (resources != null) { + forEach(resources, process); + } + } + + private List getResources() { + if (resources == null) { + resources = new ArrayList<>(2); + } + return resources; + } + + public static void forEach(List list, BiConsumer process) { + for (int i = 0; i < list.size() - 1; i += 2) { + String key = String.valueOf(list.get(i)); + Object value = list.get(i + 1); + if (value instanceof Supplier) { + value = ((Supplier)value).get(); + } + process.accept(key, String.valueOf(value)); + } + } +} diff --git a/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/StructuredEventLog.java b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/StructuredEventLog.java index b4f0a1de72a3f..0c3f8346c9289 100644 --- a/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/StructuredEventLog.java +++ b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/StructuredEventLog.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.structuredeventlog; +import org.apache.pulsar.structuredeventlog.slf4j.Slf4jStructuredEventLog; + /** * Structured event logging interface * @@ -84,6 +86,6 @@ public interface StructuredEventLog { * Create a new logger object, from which root events can be created. */ public static StructuredEventLog newLogger() { - return null; + return Slf4jStructuredEventLog.INSTANCE; } } diff --git a/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jEvent.java b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jEvent.java new file mode 100644 index 0000000000000..e26a021dc5ddd --- /dev/null +++ b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jEvent.java @@ -0,0 +1,206 @@ +/** + * 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.pulsar.structuredeventlog.slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import org.apache.pulsar.structuredeventlog.Event; +import org.apache.pulsar.structuredeventlog.EventResources; +import org.apache.pulsar.structuredeventlog.EventResourcesImpl; + +import org.slf4j.MDC; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class Slf4jEvent implements Event { + private final String id; + private String traceId = null; + private String parentId = null; + private List attributes = null; + private Level level = Level.INFO; + private Throwable throwable = null; + private final EventResourcesImpl resources; + + Slf4jEvent(EventResourcesImpl parentResources) { + this.id = randomId(); + this.resources = new EventResourcesImpl(parentResources); + } + + @Override + public Event newChildEvent() { + return new Slf4jEvent(resources).traceId(traceId).parentId(id); + } + + @Override + public Event traceId(String traceId) { + this.traceId = traceId; + return this; + } + + @Override + public Event parentId(String parentId) { + this.parentId = parentId; + return this; + } + + @Override + public Event timed() { + throw new UnsupportedOperationException("TODO"); + } + + @Override + public Event sampled(Object samplingKey, int duration, TimeUnit unit) { + throw new UnsupportedOperationException("TODO"); + } + + @Override + public Event resources(EventResources other) { + if (other instanceof EventResourcesImpl) { + this.resources.copyFrom((EventResourcesImpl)other); + } + return this; + } + + @Override + public Event resource(String key, Object value) { + resources.resource(key, value); + return this; + } + + @Override + public Event resource(String key, Supplier value) { + resources.resource(key, value); + return this; + } + + @Override + public Event attr(String key, Object value) { + getAttributes().add(key); + getAttributes().add(value); + return this; + } + + @Override + public Event attr(String key, Supplier value) { + this.attr(key, (Object)value); + return this; + } + + @Override + public Event exception(Throwable t) { + this.throwable = t; + return this; + } + + @Override + public Event atError() { + this.level = Level.ERROR; + return this; + } + + @Override + public Event atInfo() { + this.level = Level.INFO; + return this; + } + + @Override + public Event atWarn() { + this.level = Level.WARN; + return this; + } + + @Override + public void log(Enum event) { + throw new UnsupportedOperationException("TODO"); + } + + @Override + public void log(String event) { + try { + MDC.put("id", id); + if (traceId != null) { + MDC.put("traceId", traceId); + } + if (parentId != null) { + MDC.put("parentId", parentId); + } + resources.forEach(MDC::put); + if (attributes != null) { + EventResourcesImpl.forEach(attributes, MDC::put); + } + Logger logger = LoggerFactory.getLogger("stevlog"); + switch (level) { + case ERROR: + if (throwable != null) { + logger.error(event, throwable); + } else { + logger.error(event); + } + break; + case WARN: + if (throwable != null) { + logger.warn(event, throwable); + } else { + logger.warn(event); + } + break; + case INFO: + default: + if (throwable != null) { + logger.info(event, throwable); + } else { + logger.info(event); + } + break; + } + } finally { + MDC.clear(); + } + } + + @Override + public void stash() { + throw new UnsupportedOperationException("TODO"); + } + + private List getAttributes() { + if (attributes == null) { + attributes = new ArrayList<>(); + } + return attributes; + } + + static String randomId() { + return Long.toString( + ThreadLocalRandom.current().nextLong(0x100000000000000L, + 0xFFFFFFFFFFFFFFFL), + 16); + } + + enum Level { + INFO, + WARN, + ERROR + } +} diff --git a/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jStructuredEventLog.java b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jStructuredEventLog.java new file mode 100644 index 0000000000000..c0fb857685b0e --- /dev/null +++ b/structured-event-log/src/main/java/org/apache/pulsar/structuredeventlog/slf4j/Slf4jStructuredEventLog.java @@ -0,0 +1,43 @@ +/** + * 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.pulsar.structuredeventlog.slf4j; + +import org.apache.pulsar.structuredeventlog.Event; +import org.apache.pulsar.structuredeventlog.EventResources; +import org.apache.pulsar.structuredeventlog.EventResourcesImpl; +import org.apache.pulsar.structuredeventlog.StructuredEventLog; + +public class Slf4jStructuredEventLog implements StructuredEventLog { + public static Slf4jStructuredEventLog INSTANCE = new Slf4jStructuredEventLog(); + + @Override + public Event newRootEvent() { + return new Slf4jEvent(null).traceId(Slf4jEvent.randomId()); + } + + @Override + public EventResources newEventResources() { + return new EventResourcesImpl(null); + } + + @Override + public Event unstash() { + throw new UnsupportedOperationException("TODO"); + } +} diff --git a/structured-event-log/src/test/java/org/apache/pulsar/structuredeventlog/StructuredEventLogTest.java b/structured-event-log/src/test/java/org/apache/pulsar/structuredeventlog/StructuredEventLogTest.java new file mode 100644 index 0000000000000..b39d01ef8d2f4 --- /dev/null +++ b/structured-event-log/src/test/java/org/apache/pulsar/structuredeventlog/StructuredEventLogTest.java @@ -0,0 +1,359 @@ +/** + * 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.pulsar.structuredeventlog; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.BufferedReader; +import java.io.StringReader; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.LoggerContext; +import org.apache.logging.log4j.core.appender.WriterAppender; +import org.apache.logging.log4j.core.config.Configuration; +import org.apache.logging.log4j.core.config.LoggerConfig ; +import org.apache.logging.log4j.core.layout.JsonLayout; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class StructuredEventLogTest { + private final static String APPENDER_NAME = "stevlogtest"; + StringWriter writer; + + @BeforeMethod + public void setupLog4j() throws Exception { + LoggerContext context = LoggerContext.getContext(false); + Configuration config = context.getConfiguration(); + + writer = new StringWriter(); + + + LoggerConfig logConfig = config.getLoggerConfig(APPENDER_NAME); + for (Map.Entry a : logConfig.getAppenders().entrySet()) { + if (a.getKey().equals(APPENDER_NAME)) { + a.getValue().stop(); + } + } + logConfig.removeAppender(APPENDER_NAME); + + JsonLayout layout = JsonLayout.newBuilder().setEventEol(true).setCompact(true).setProperties(true).build(); + Appender appender = WriterAppender.createAppender(layout, null, writer, "stevlogtest", false, true); + appender.start(); + logConfig.addAppender(appender, null, null); + logConfig.setLevel(Level.DEBUG); + } + + @Test + public void testTraceId() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + Event e = log.newRootEvent(); + e.newChildEvent().log("child"); + e.log("parent"); + + log.newRootEvent().log("second"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("child")); + assertThat(logged.get(1).get("message"), equalTo("parent")); + assertThat(logged.get(2).get("message"), equalTo("second")); + + assertThat(contextMapField(logged.get(0), "traceId"), + equalTo(contextMapField(logged.get(1), "traceId"))); + assertThat(contextMapField(logged.get(0), "traceId"), + not(equalTo(contextMapField(logged.get(2), "traceId")))); + } + + @Test + public void testParentId() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + Event e1 = log.newRootEvent(); + Event e2 = e1.newChildEvent(); + e2.newChildEvent().log("child2"); + e2.log("child1"); + e1.log("parent"); + + log.newRootEvent().log("second"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("child2")); + assertThat(logged.get(1).get("message"), equalTo("child1")); + assertThat(logged.get(2).get("message"), equalTo("parent")); + assertThat(logged.get(3).get("message"), equalTo("second")); + + assertThat(contextMapField(logged.get(0), "parentId"), not(nullValue())); + assertThat(contextMapField(logged.get(0), "parentId"), + equalTo(contextMapField(logged.get(1), "id"))); + assertThat(contextMapField(logged.get(1), "parentId"), + equalTo(contextMapField(logged.get(2), "id"))); + assertThat(contextMapField(logged.get(2), "parentId"), nullValue()); + assertThat(contextMapField(logged.get(3), "parentId"), nullValue()); + } + + @Test + public void testResources() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + EventResources res = log.newEventResources() + .resource("r1", "v1") + .resource("r2", () -> "v2"); + Event e1 = log.newRootEvent() + .resources(res) + .resource("r3", "v3") + .resource("r4", () -> "v4"); + Event e2 = e1.newChildEvent().resource("r5", "v5"); + e2.newChildEvent().resource("r6", "v6").log("child2"); + e2.log("child1"); + e1.log("parent"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("child2")); + assertThat(logged.get(1).get("message"), equalTo("child1")); + assertThat(logged.get(2).get("message"), equalTo("parent")); + + assertThat(contextMapField(logged.get(0), "r6"), equalTo("v6")); + assertThat(contextMapField(logged.get(0), "r5"), equalTo("v5")); + assertThat(contextMapField(logged.get(0), "r4"), equalTo("v4")); + assertThat(contextMapField(logged.get(0), "r3"), equalTo("v3")); + assertThat(contextMapField(logged.get(0), "r2"), equalTo("v2")); + assertThat(contextMapField(logged.get(0), "r1"), equalTo("v1")); + + assertThat(contextMapField(logged.get(1), "r6"), nullValue()); + assertThat(contextMapField(logged.get(1), "r5"), equalTo("v5")); + assertThat(contextMapField(logged.get(1), "r4"), equalTo("v4")); + assertThat(contextMapField(logged.get(1), "r3"), equalTo("v3")); + assertThat(contextMapField(logged.get(1), "r2"), equalTo("v2")); + assertThat(contextMapField(logged.get(1), "r1"), equalTo("v1")); + + assertThat(contextMapField(logged.get(2), "r6"), nullValue()); + assertThat(contextMapField(logged.get(2), "r5"), nullValue()); + assertThat(contextMapField(logged.get(2), "r4"), equalTo("v4")); + assertThat(contextMapField(logged.get(2), "r3"), equalTo("v3")); + assertThat(contextMapField(logged.get(2), "r2"), equalTo("v2")); + assertThat(contextMapField(logged.get(2), "r1"), equalTo("v1")); + } + + @Test + public void testResourcesNullTest() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + EventResources res = log.newEventResources() + .resource(null, "v1") + .resource("r1", null) + .resource("r2", () -> null); + Event e1 = log.newRootEvent() + .resources(res) + .resource(null, "v2") + .resource("r3", null) + .resource("r4", () -> null); + e1.newChildEvent() + .resource(null, "v3") + .resource("r5", null) + .log("child1"); + e1.log("parent"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("child1")); + assertThat(logged.get(1).get("message"), equalTo("parent")); + + assertThat(contextMapField(logged.get(0), "r5"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "r4"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "r3"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "r2"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "r1"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "null"), equalTo("v3")); + + assertThat(contextMapField(logged.get(1), "r5"), nullValue()); + assertThat(contextMapField(logged.get(1), "r4"), equalTo("null")); + assertThat(contextMapField(logged.get(1), "r3"), equalTo("null")); + assertThat(contextMapField(logged.get(1), "r2"), equalTo("null")); + assertThat(contextMapField(logged.get(1), "r1"), equalTo("null")); + assertThat(contextMapField(logged.get(1), "null"), equalTo("v2")); + } + + @Test + public void testAttributes() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + Event e1 = log.newRootEvent() + .attr("a1", "v1") + .attr("a2", () -> "v2"); + Event e2 = e1.newChildEvent().attr("a3", "v3"); + e2.newChildEvent().resource("a4", "v4").log("child2"); + e2.log("child1"); + e1.log("parent"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("child2")); + assertThat(logged.get(1).get("message"), equalTo("child1")); + assertThat(logged.get(2).get("message"), equalTo("parent")); + + assertThat(contextMapField(logged.get(0), "a4"), equalTo("v4")); + assertThat(contextMapField(logged.get(0), "a3"), nullValue()); + assertThat(contextMapField(logged.get(0), "a2"), nullValue()); + assertThat(contextMapField(logged.get(0), "a1"), nullValue()); + + assertThat(contextMapField(logged.get(1), "a4"), nullValue()); + assertThat(contextMapField(logged.get(1), "a3"), equalTo("v3")); + assertThat(contextMapField(logged.get(1), "a2"), nullValue()); + assertThat(contextMapField(logged.get(1), "a1"), nullValue()); + + assertThat(contextMapField(logged.get(2), "a4"), nullValue()); + assertThat(contextMapField(logged.get(2), "a3"), nullValue()); + assertThat(contextMapField(logged.get(2), "a2"), equalTo("v2")); + assertThat(contextMapField(logged.get(2), "a1"), equalTo("v1")); + } + + @Test + public void testAttributedNullTest() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent() + .attr(null, "v1") + .attr("a1", null) + .attr("a2", () -> null) + .log("msg"); + + log.newRootEvent() + .attr(null, "v1") + .attr("a1", null) + .attr("a2", () -> null) + .log("msg"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("msg")); + + assertThat(contextMapField(logged.get(0), "null"), equalTo("v1")); + assertThat(contextMapField(logged.get(0), "a1"), equalTo("null")); + assertThat(contextMapField(logged.get(0), "a2"), equalTo("null")); + } + + @Test + public void testInfoLevel() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().log("info1"); + log.newRootEvent().atInfo().log("info2"); + log.newRootEvent().atError().atInfo().log("info3"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("info1")); + assertThat(logged.get(1).get("message"), equalTo("info2")); + assertThat(logged.get(2).get("message"), equalTo("info3")); + + assertThat(logged.get(0).get("level"), equalTo("INFO")); + assertThat(logged.get(1).get("level"), equalTo("INFO")); + assertThat(logged.get(2).get("level"), equalTo("INFO")); + } + + @SuppressWarnings("unchecked") + @Test + public void testInfoLevelException() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().exception(new Throwable("cause1")).log("info1"); + log.newRootEvent().atInfo().exception(new Throwable("cause2")).log("info2"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("info1")); + + assertThat(((Map)logged.get(0).get("thrown")).get("message"), equalTo("cause1")); + assertThat(logged.get(1).get("message"), equalTo("info2")); + assertThat(((Map)logged.get(1).get("thrown")).get("message"), equalTo("cause2")); + } + + @Test + public void testWarnLevel() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().atWarn().log("warn1"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("warn1")); + assertThat(logged.get(0).get("level"), equalTo("WARN")); + } + + @SuppressWarnings("unchecked") + @Test + public void testWarnLevelException() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().atWarn().exception(new Throwable("cause1")).log("warn1"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("warn1")); + assertThat(((Map)logged.get(0).get("thrown")).get("message"), equalTo("cause1")); + } + + @Test + public void testErrorLevel() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().atError().log("error1"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("error1")); + assertThat(logged.get(0).get("level"), equalTo("ERROR")); + } + + @SuppressWarnings("unchecked") + @Test + public void testErrorLevelException() throws Exception { + StructuredEventLog log = StructuredEventLog.newLogger(); + + log.newRootEvent().atError().exception(new Throwable("cause1")).log("error1"); + + List> logged = getLogged(); + assertThat(logged.get(0).get("message"), equalTo("error1")); + assertThat(((Map)logged.get(0).get("thrown")).get("message"), equalTo("cause1")); + } + + + + @SuppressWarnings("unchecked") + private Object contextMapField(Map map, String field) { + return ((Map)map.get("contextMap")).get(field); + } + + @SuppressWarnings("unchecked") + private List> getLogged() throws Exception { + List> logged = new ArrayList<>(); + ObjectMapper o = new ObjectMapper(); + try (BufferedReader r = new BufferedReader(new StringReader(writer.toString()))) { + String line = r.readLine(); + while (line != null) { + Map log = o.readValue(line, Map.class); + if (log.get("loggerName").toString().startsWith("stevlog")) { + logged.add(log); + } + line = r.readLine(); + } + } + return logged; + } +} +