You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2017/02/24 20:53:24 UTC

[1/3] incubator-metron git commit: METRON-690: Create a DSL-based timestamp lookup for profiler to enable sparse windows closes apache/incubator-metron#450

Repository: incubator-metron
Updated Branches:
  refs/heads/master 7abd7e8a2 -> 84d347195


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/IntervalPredicateTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/IntervalPredicateTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/IntervalPredicateTest.java
new file mode 100644
index 0000000..8a0ea62
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/IntervalPredicateTest.java
@@ -0,0 +1,89 @@
+/*
+ *
+ *  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.metron.profiler.client.stellar;
+
+import org.apache.commons.lang3.Range;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+public class IntervalPredicateTest {
+  @Test
+  public void testBasicTest() {
+    List<Range<Long>> intervals = new ArrayList<Range<Long>>() {{
+      add(Range.between(0L, 10L));
+      add(Range.between(20L, 30L));
+      add(Range.between(40L, 50L));
+    }};
+    IntervalPredicate<Long> predicate = new IntervalPredicate.Identity(intervals);
+    Assert.assertTrue(predicate.test(0L));
+    Assert.assertTrue(predicate.test(10L));
+    Assert.assertTrue(predicate.test(5L));
+    Assert.assertFalse(predicate.test(51L));
+    Assert.assertFalse(predicate.test(15L));
+  }
+
+  @Test
+  public void testWithOverlap() {
+    List<Range<Long>> intervals = new ArrayList<Range<Long>>() {{
+      add(Range.between(0L, 10L));
+      add(Range.between(5L, 30L));
+      add(Range.between(40L, 50L));
+    }};
+    IntervalPredicate<Long> predicate = new IntervalPredicate.Identity(intervals);
+    Assert.assertTrue(predicate.test(0L));
+    Assert.assertTrue(predicate.test(5L));
+    Assert.assertTrue(predicate.test(30L));
+    Assert.assertTrue(predicate.test(10L));
+    Assert.assertFalse(predicate.test(51L));
+    Assert.assertTrue(predicate.test(15L));
+    Assert.assertFalse(predicate.test(31L));
+    Assert.assertTrue(predicate.test(45L));
+  }
+
+  @Test
+  public void testTrivialCase() {
+    List<Range<Long>> intervals = new ArrayList<Range<Long>>() {{
+      add(Range.between(0L, 10L));
+    }};
+    IntervalPredicate<Long> predicate = new IntervalPredicate.Identity(intervals);
+    Assert.assertTrue(predicate.test(0L));
+    Assert.assertTrue(predicate.test(5L));
+    Assert.assertTrue(predicate.test(10L));
+    Assert.assertFalse(predicate.test(51L));
+    Assert.assertFalse(predicate.test(15L));
+  }
+
+  @Test
+  public void testDegenerateCase() {
+    List<Range<Long>> intervals = new ArrayList<Range<Long>>() {{
+      add(Range.between(10L, 10L));
+    }};
+    IntervalPredicate<Long> predicate = new IntervalPredicate.Identity(intervals);
+    Assert.assertFalse(predicate.test(0L));
+    Assert.assertFalse(predicate.test(5L));
+    Assert.assertTrue(predicate.test(10L));
+    Assert.assertFalse(predicate.test(11L));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/WindowLookbackTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/WindowLookbackTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/WindowLookbackTest.java
new file mode 100644
index 0000000..aafbe5c
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/WindowLookbackTest.java
@@ -0,0 +1,207 @@
+/*
+ *
+ *  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.metron.profiler.client.stellar;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.Range;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.functions.resolver.FunctionResolver;
+import org.apache.metron.common.dsl.functions.resolver.SimpleFunctionResolver;
+import org.apache.metron.common.stellar.StellarProcessor;
+import org.apache.metron.profiler.ProfilePeriod;
+import org.apache.metron.profiler.client.window.WindowProcessor;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+
+public class WindowLookbackTest {
+
+  static FunctionResolver resolver;
+  static Context context;
+  @BeforeClass
+  public static void setup() {
+    resolver = new SimpleFunctionResolver()
+                    .withClass(GetProfile.class)
+                    .withClass(FixedLookback.class)
+                    .withClass(WindowLookback.class);
+    context = new Context.Builder()
+                    .with(Context.Capabilities.GLOBAL_CONFIG, () -> new HashMap<>())
+                    .build();
+  }
+
+  @Test
+  public void testSpecifyingConfig() throws Exception {
+    //we should be able to specify the config and have it take hold.  If we change the
+    //profile duration to 1 minute instead of 15 minutes (the default), then we should see
+    //the correct number of profiles.
+    long durationMs = 60000;
+    State state = test("1 hour", new Date()
+                      , Optional.of(
+                              ImmutableMap.of( ProfilerConfig.PROFILER_PERIOD.getKey(), 1 )
+                                   )
+                      ,Assertions.NOT_EMPTY,Assertions.CONTIGUOUS);
+    Assert.assertEquals(TimeUnit.HOURS.toMillis(1) / durationMs, state.periods.size());
+  }
+
+  @Test
+  public void testSpecifyingOnlySelector() {
+    String stellarStatement = "PROFILE_WINDOW('1 hour')";
+    Map<String, Object> variables = new HashMap<>();
+    StellarProcessor stellar = new StellarProcessor();
+    List<ProfilePeriod> periods = (List<ProfilePeriod>)stellar.parse( stellarStatement
+                                                                    , k -> variables.get(k)
+                                                                    , resolver
+                                                                    , context
+                                                                    );
+    Assert.assertEquals(TimeUnit.HOURS.toMillis(1) / getDurationMs(), periods.size());
+  }
+
+  @Test
+  public void testDenseLookback() throws Exception {
+    State state = test("1 hour", Assertions.NOT_EMPTY, Assertions.CONTIGUOUS);
+    Assert.assertEquals(TimeUnit.HOURS.toMillis(1) / getDurationMs(), state.periods.size());
+  }
+
+  @Test
+  public void testShiftedDenseLookback() throws Exception {
+    State state = test("from 2 hours ago to 30 minutes ago", Assertions.NOT_EMPTY
+                                                           , Assertions.CONTIGUOUS
+                                                           , Assertions.INTERVALS_CONTAIN_ALL_PERIODS
+                                                           );
+    Assert.assertEquals(TimeUnit.MINUTES.toMillis(90) / getDurationMs(), state.periods.size());
+  }
+
+  @Test
+  public void testShiftedSparseLookback() throws Exception {
+    State state = test("30 minute window every 1 hour from 2 hours ago to 30 minutes ago", Assertions.NOT_EMPTY
+                                                                                         , Assertions.DISCONTIGUOUS
+                                                                                         , Assertions.INTERVALS_CONTAIN_ALL_PERIODS
+                                                                                         );
+    Assert.assertEquals(TimeUnit.MINUTES.toMillis(60) / getDurationMs(), state.periods.size());
+  }
+
+  @Test
+  public void testEmptyDueToExclusions() throws Exception {
+    test("30 minute window every 24 hours from 7 days ago including saturdays excluding weekends", Assertions.EMPTY);
+  }
+
+  @Test(expected= ParseException.class)
+  public void testErrorInSelector() throws Exception {
+    test("30 minute idow every 24 hours from 7 days ago including saturdays excluding weekends", Assertions.EMPTY);
+  }
+
+  long getDurationMs() {
+    int duration = ProfilerConfig.PROFILER_PERIOD.getDefault(Integer.class);
+    TimeUnit unit = TimeUnit.valueOf(ProfilerConfig.PROFILER_PERIOD_UNITS.getDefault(String.class));
+    return unit.toMillis(duration);
+  }
+
+  public State test(String windowSelector, Assertions... assertions) {
+    return test(windowSelector, new Date(), Optional.empty(), assertions);
+  }
+
+  public State test(String windowSelector, Date now, Optional<Map<String, Object>> config, Assertions... assertions) {
+
+    List<Range<Long>> windowIntervals = WindowProcessor.process(windowSelector).toIntervals(now.getTime());
+    String stellarStatement = "PROFILE_WINDOW('" + windowSelector + "', now"
+                            + (config.isPresent()?", config":"")
+                            + ")";
+    Map<String, Object> variables = new HashMap<>();
+    variables.put("now", now.getTime());
+    if(config.isPresent()) {
+      variables.put("config", config.get());
+    }
+    StellarProcessor stellar = new StellarProcessor();
+    List<ProfilePeriod> periods = (List<ProfilePeriod>)stellar.parse( stellarStatement
+                                                                    , k -> variables.get(k)
+                                                                    , resolver
+                                                                    , context
+                                                                    );
+    State state = new State(windowIntervals, periods);
+    for(Assertions assertion : assertions) {
+      Assert.assertTrue(assertion.name(), assertion.test(state));
+    }
+    return state;
+  }
+
+  private enum Assertions implements Predicate<State>{
+    EMPTY( state -> state.windowIntervals.isEmpty() && state.periods.isEmpty() ),
+    NOT_EMPTY( state -> !state.windowIntervals.isEmpty() && !state.periods.isEmpty()),
+    CONTIGUOUS( state -> {
+      if(state.periods.size() < 2) {
+        return true;
+      }
+      long duration = state.periods.get(1).getStartTimeMillis() - state.periods.get(0).getStartTimeMillis();
+      for(int i = 1;i < state.periods.size();++i) {
+        long left = state.periods.get(i - 1).getStartTimeMillis();
+        long right = state.periods.get(i).getStartTimeMillis();
+        if(right - left != duration) {
+          return false;
+        }
+      }
+      return true;
+    }),
+    DISCONTIGUOUS( state -> !Assertions.CONTIGUOUS.test(state)),
+    INTERVALS_CONTAIN_ALL_PERIODS( state -> {
+      List<Range<Long>> windowIntervals = state.windowIntervals;
+      List<ProfilePeriod> periods = state.periods;
+
+      Set<Range<Long>> foundIntervals = new HashSet<>();
+      for(ProfilePeriod period : periods) {
+        boolean found = false;
+        for(Range<Long> interval : windowIntervals) {
+          if(interval.contains(period.getStartTimeMillis())) {
+            foundIntervals.add(interval);
+            found = true;
+          }
+        }
+        if(!found) {
+          return false;
+        }
+      }
+      return foundIntervals.size() == windowIntervals.size();
+    })
+    ;
+    Predicate<State> predicate;
+    Assertions(Predicate<State> predicate) {
+      this.predicate = predicate;
+    }
+
+    @Override
+    public boolean test(State s) {
+      return predicate.test(s);
+    }
+  }
+
+  private static class State {
+    List<Range<Long>> windowIntervals;
+    List<ProfilePeriod> periods;
+    public State(List<Range<Long>> windowIntervals, List<ProfilePeriod> periods) {
+      this.periods = periods;
+      this.windowIntervals = windowIntervals;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/window/WindowProcessorTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/window/WindowProcessorTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/window/WindowProcessorTest.java
new file mode 100644
index 0000000..94f9e04
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/window/WindowProcessorTest.java
@@ -0,0 +1,314 @@
+/*
+ *
+ *  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.metron.profiler.client.window;
+
+import org.apache.commons.lang3.Range;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class WindowProcessorTest {
+
+  @Test
+  public void testBaseCase() {
+    for (String text : new String[] {
+            "1 hour"
+            ,"1 hour(s)"
+            ,"1 hours"
+    }) {
+      Window w = WindowProcessor.process(text);
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(1, intervals.size());
+      Assert.assertEquals(now.getTime(), (long)intervals.get(0).getMaximum());
+      Assert.assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(1), (long)intervals.get(0).getMinimum());
+    }
+  }
+
+  @Test
+  public void testDenseWindow() {
+    for (String text : new String[] {
+            "from 2 hours ago to 30 minutes ago"
+            ,"starting from 2 hours until 30 minutes"
+            ,"starting from 2 hours ago until 30 minutes ago"
+            ,"starting from 30 minutes ago until 2 hours ago"
+            ,"from 30 minutes ago to 2 hours ago "
+    }) {
+      Window w = WindowProcessor.process(text);
+    /*
+    A dense window starting 2 hour ago and continuing until 30 minutes ago
+     */
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(1, intervals.size());
+      assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(2), intervals.get(0).getMinimum());
+      assertEquals(now.getTime() - TimeUnit.MINUTES.toMillis(30), intervals.get(0).getMaximum());
+    }
+  }
+
+  @Test
+  public void testSparse() {
+    for(String text : new String[] {
+      "30 minute window every 1 hour from 2 hours ago to 30 minutes ago",
+      "30 minute window every 1 hour starting from 2 hours ago to 30 minutes ago",
+      "30 minute window every 1 hour starting from 2 hours ago until 30 minutes ago",
+      "30 minute window for every 1 hour starting from 2 hours ago until 30 minutes ago",
+    })
+    {
+      Window w = WindowProcessor.process(text);
+    /*
+    A window size of 30 minutes
+    Starting 2 hour ago and continuing until 30 minutes ago
+    window 1: ( now - 2 hour, now - 2 hour + 30 minutes)
+    window 2: (now - 1 hour, now - 1 hour + 30 minutes)
+     */
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(2, intervals.size());
+      assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(2), intervals.get(0).getMinimum());
+      assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(2) + TimeUnit.MINUTES.toMillis(30), intervals.get(0).getMaximum());
+      assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(1), intervals.get(1).getMinimum());
+      assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(1) + TimeUnit.MINUTES.toMillis(30), intervals.get(1).getMaximum());
+    }
+  }
+
+
+  @Test
+  public void testRepeatTilNow() {
+    Window w = WindowProcessor.process("30 minute window every 1 hour from 3 hours ago");
+    /*
+    A window size of 30 minutes
+    Starting 3 hours ago and continuing until now
+    window 1: ( now - 3 hour, now - 3 hour + 30 minutes)
+    window 2: ( now - 2 hour, now - 2 hour + 30 minutes)
+    window 3: ( now - 1 hour, now - 1 hour + 30 minutes)
+     */
+    Date now = new Date();
+    List<Range<Long>> intervals = w.toIntervals(now.getTime());
+    Assert.assertEquals(3, intervals.size());
+
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(3), intervals.get(0).getMinimum());
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(3) + TimeUnit.MINUTES.toMillis(30), intervals.get(0).getMaximum());
+
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(2), intervals.get(1).getMinimum());
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(2) + TimeUnit.MINUTES.toMillis(30), intervals.get(1).getMaximum());
+
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(1), intervals.get(2).getMinimum());
+    assertEquals(now.getTime() - TimeUnit.HOURS.toMillis(1) + TimeUnit.MINUTES.toMillis(30), intervals.get(2).getMaximum());
+  }
+
+  @Test
+  public void testRepeatWithInclusions() {
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago including tuesdays");
+    /*
+    A window size of 30 minutes
+    Starting 14 days ago  and continuing until now
+    Gotta be 2 tuesdays in 14 days.
+     */
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(2, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago including this day of the week");
+    /*
+    A window size of 30 minutes
+    Starting 14 days ago  and continuing until now
+    Gotta be 2 days with the same dow in 14 days.
+     */
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(2, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago");
+    /*
+    A window size of 30 minutes
+    Starting 14 days ago  and continuing until now
+    Gotta be 14 intervals in 14 days.
+     */
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(14, intervals.size());
+    }
+  }
+
+
+  @Test
+  public void testRepeatWithConflictingExclusionInclusion() throws ParseException {
+    Window w = WindowProcessor.process("30 minute window every 24 hours from 7 days ago including saturdays excluding weekends");
+
+    Date now = new Date();
+    List<Range<Long>> intervals = w.toIntervals(now.getTime());
+    Assert.assertEquals(0, intervals.size());
+  }
+
+  @Test
+  public void testRepeatWithWeekendExclusion() throws ParseException {
+    Window w = WindowProcessor.process("30 minute window every 24 hours from 7 days ago excluding weekends");
+
+    Date now = new Date();
+    List<Range<Long>> intervals = w.toIntervals(now.getTime());
+    Assert.assertEquals(5, intervals.size());
+  }
+
+  @Test
+  public void testRepeatWithInclusionExclusion() throws ParseException {
+    Window w = WindowProcessor.process("30 minute window every 24 hours from 7 days ago including holidays:us excluding weekends");
+
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+    Date now = sdf.parse("2017/12/26 12:00");
+    List<Range<Long>> intervals = w.toIntervals(now.getTime());
+    Assert.assertEquals(1, intervals.size());
+  }
+
+  @Test
+  public void testManyMoonsAgo() throws ParseException {
+    {
+      Window w = WindowProcessor.process("1 hour window every 24 hours starting from 56 days ago");
+
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(56, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("1 hour window every 24 hours starting from 56 days ago including this day of the week");
+
+      Date now = new Date();
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(8, intervals.size());
+    }
+  }
+
+  @Test
+  public void testRepeatWithWeekdayExclusion() throws ParseException {
+    Window w = WindowProcessor.process("30 minute window every 24 hours from 7 days ago excluding weekdays");
+
+    Date now = new Date();
+    List<Range<Long>> intervals = w.toIntervals(now.getTime());
+    Assert.assertEquals(2, intervals.size());
+  }
+
+  @Test
+  public void testRepeatWithHolidayExclusion() throws ParseException {
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago excluding holidays:us");
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/12/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(13, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago excluding holidays:us:nyc");
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/12/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(13, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago excluding holidays:us");
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/08/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(14, intervals.size());
+    }
+  }
+
+  @Test
+  public void testDateDaySpecifier() throws ParseException {
+    for(String text : new String[] {
+        "30 minute window every 24 hours from 14 days ago including date:20171225:yyyyMMdd",
+        "30 minute window every 24 hours from 14 days ago including date:2017-12-25:yyyy-MM-dd",
+        "30 minute window every 24 hours from 14 days ago including date:2017/12/25",
+      })
+    {
+      Window w = WindowProcessor.process(text);
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/12/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(1, intervals.size());
+      Date includedDate = new Date(intervals.get(0).getMinimum());
+      SimpleDateFormat equalityFormat = new SimpleDateFormat("yyyyMMdd");
+      Assert.assertEquals("20171225", equalityFormat.format(includedDate));
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago excluding date:2017/12/25");
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/12/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(13, intervals.size());
+    }
+    {
+      Window w = WindowProcessor.process("30 minute window every 24 hours from 14 days ago including date:2017/12/25, date:2017/12/24");
+      SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
+      Date now = sdf.parse("2017/12/26 12:00");
+      List<Range<Long>> intervals = w.toIntervals(now.getTime());
+      Assert.assertEquals(2, intervals.size());
+      {
+        Date includedDate = new Date(intervals.get(0).getMinimum());
+        SimpleDateFormat equalityFormat = new SimpleDateFormat("yyyyMMdd");
+        Assert.assertEquals("20171224", equalityFormat.format(includedDate));
+      }
+      {
+        Date includedDate = new Date(intervals.get(1).getMinimum());
+        SimpleDateFormat equalityFormat = new SimpleDateFormat("yyyyMMdd");
+        Assert.assertEquals("20171225", equalityFormat.format(includedDate));
+      }
+    }
+  }
+
+  @Test(expected=org.apache.metron.common.dsl.ParseException.class)
+  public void testWithInvalidDaySpecifier() throws ParseException {
+    WindowProcessor.process("30 minute window every 24 hours from 14 days ago excluding hoolidays:us");
+  }
+
+  @Test(expected=org.apache.metron.common.dsl.ParseException.class)
+  public void testWithInvalidTimeUnit() throws ParseException {
+    WindowProcessor.process("30 minute window every 24 months from 14 days ago");
+  }
+
+  @Test(expected=org.apache.metron.common.dsl.ParseException.class)
+  public void testWithInvalidWindowUnit() throws ParseException {
+    WindowProcessor.process("30 minuete window every 24 hours from 14 days ago");
+  }
+
+  @Test(expected=org.apache.metron.common.dsl.ParseException.class)
+  public void testWithInvalidTimeNumber() throws ParseException {
+    WindowProcessor.process("30p minute window every 24 hours from 14 days ago");
+  }
+
+  @Test(expected=org.apache.metron.common.dsl.ParseException.class)
+  public void testInvalidDaySpecifier() throws ParseException {
+    WindowProcessor.process("30 minute window every 14 hours from 14 days ago including date");
+  }
+
+  private static void assertEquals(long expected, long actual) {
+    long diff = expected - actual;
+    long diffInMinutes = TimeUnit.MILLISECONDS.toMinutes(diff);
+    String message =  expected + " - " + actual + " = " + diffInMinutes + " minutes off.";
+    Assert.assertEquals(message, expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
index f916d65..2f7f356 100644
--- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
@@ -111,12 +111,12 @@ public class ProfilePeriod {
   }
 
   public static <T> List<T> visitPeriods(long startEpochMillis
-                                           , long endEpochMillis
-                                           , long duration
-                                           , TimeUnit units
-                                           , Optional<Predicate<ProfilePeriod>> inclusionPredicate
-                                           , Function<ProfilePeriod,T> transformation
-                                           )
+                                        , long endEpochMillis
+                                        , long duration
+                                        , TimeUnit units
+                                        , Optional<Predicate<ProfilePeriod>> inclusionPredicate
+                                        , Function<ProfilePeriod,T> transformation
+                                        )
   {
     ProfilePeriod period = new ProfilePeriod(startEpochMillis, duration, units);
     List<T> ret = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-platform/metron-common/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md
index 067bf8d..4ffb8f1 100644
--- a/metron-platform/metron-common/README.md
+++ b/metron-platform/metron-common/README.md
@@ -122,6 +122,7 @@ The `!=` operator is the negation of the above.
 | [ `MONTH`](#month)                                                                                 |
 | [ `PROFILE_GET`](#profile_get)                                                                     |
 | [ `PROFILE_FIXED`](#profile_fixed)                                                                     |
+| [ `PROFILE_WINDOW`](#profile_window)                                                                     |
 | [ `PROTOCOL_TO_NAME`](#protocol_to_name)                                                           |
 | [ `REGEXP_MATCH`](#regexp_match)                                                                   |
 | [ `SPLIT`](#split)                                                                                 |
@@ -450,6 +451,14 @@ The `!=` operator is the negation of the above.
     * config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter of the same name. Default is the empty Map, meaning no overrides.
   * Returns: The selected profile measurement timestamps.  These are ProfilePeriod objects.
 
+### `PROFILE_WINDOW`
+  * Description: The profiler periods associated with a window selector statement from an optional reference timestamp.
+  * Input:
+    * windowSelector - The statement specifying the window to select.
+    * now - Optional - The timestamp to use for now.
+    * config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter of the same name. Default is the empty Map, meaning no overrides.
+  * Returns: The selected profile measurement periods.  These are ProfilePeriod objects.
+
 ### `PROTOCOL_TO_NAME`
   * Description: Converts the IANA protocol number to the protocol name
   * Input:

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-platform/metron-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml
index 9c9e761..5d4c2fd 100644
--- a/metron-platform/metron-common/pom.xml
+++ b/metron-platform/metron-common/pom.xml
@@ -27,7 +27,6 @@
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <commons.config.version>1.10</commons.config.version>
-        <antlr.version>4.5</antlr.version>
     </properties>
     <repositories>
         <repository>
@@ -65,7 +64,7 @@
         <dependency>
             <groupId>org.antlr</groupId>
             <artifactId>antlr4-runtime</artifactId>
-            <version>${antlr.version}</version>
+            <version>${global_antlr_version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
@@ -376,7 +375,7 @@
             <plugin>
                 <groupId>org.antlr</groupId>
                 <artifactId>antlr4-maven-plugin</artifactId>
-                <version>${antlr.version}</version>
+                <version>${global_antlr_version}</version>
                 <configuration>
                   <outputDirectory>${basedir}/src/main/java</outputDirectory>
                 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/GrammarUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/GrammarUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/GrammarUtils.java
new file mode 100644
index 0000000..e65af94
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/GrammarUtils.java
@@ -0,0 +1,133 @@
+/*
+ *
+ *  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.metron.common.dsl;
+
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.antlr.v4.runtime.Token;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class GrammarUtils {
+
+  public static String toSyntaxTree(ParseTree tree) {
+    return new AST(tree).toString();
+  }
+
+  /**
+   * This is a utility class to walk the parse tree for an antlr ParseTree
+   */
+  private static class AST {
+
+    private final Object payload;
+    private final List<AST> children;
+
+    public AST(ParseTree tree) {
+      this(null, tree);
+    }
+
+    private AST(AST ast, ParseTree tree) {
+      this(ast, tree, new ArrayList<>());
+    }
+
+    private AST(AST parent, ParseTree tree, List<AST> children) {
+      this.payload = getPayload(tree);
+      this.children = children;
+      if (parent == null) {
+        walk(tree, this);
+      }
+      else {
+        parent.children.add(this);
+      }
+    }
+
+    private Object getPayload(ParseTree tree) {
+      if (tree.getChildCount() == 0) {
+        return tree.getPayload();
+      }
+      else {
+        String ruleName = tree.getClass().getSimpleName().replace("Context", "");
+        return Character.toLowerCase(ruleName.charAt(0)) + ruleName.substring(1);
+      }
+    }
+
+    private static void walk(ParseTree tree, AST ast) {
+      if (tree.getChildCount() == 0) {
+        new AST(ast, tree);
+      }
+      else if (tree.getChildCount() == 1) {
+        walk(tree.getChild(0), ast);
+      }
+      else if (tree.getChildCount() > 1) {
+        for (int i = 0; i < tree.getChildCount(); i++) {
+          AST temp = new AST(ast, tree.getChild(i));
+          if (!(temp.payload instanceof Token)) {
+            walk(tree.getChild(i), temp);
+          }
+        }
+      }
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      AST ast = this;
+      List<AST> firstStack = new ArrayList<>();
+      firstStack.add(ast);
+      List<List<AST>> childListStack = new ArrayList<>();
+      childListStack.add(firstStack);
+      while (!childListStack.isEmpty()) {
+        List<AST> childStack = childListStack.get(childListStack.size() - 1);
+        if (childStack.isEmpty()) {
+          childListStack.remove(childListStack.size() - 1);
+        }
+        else {
+          ast = childStack.remove(0);
+          String caption;
+          if (ast.payload instanceof Token) {
+            Token token = (Token) ast.payload;
+            caption = String.format("TOKEN[type: %s, text: %s]",
+                    token.getType(), token.getText().replace("\n", "\\n"));
+          }
+          else {
+            caption = String.valueOf(ast.payload);
+          }
+          String indent = "";
+          for (int i = 0; i < childListStack.size() - 1; i++) {
+            indent += (childListStack.get(i).size() > 0) ? "|  " : "   ";
+          }
+          builder.append(indent)
+                  .append(childStack.isEmpty() ? "'- " : "|- ")
+                  .append(caption)
+                  .append("\n");
+
+          if (ast.children.size() > 0) {
+            List<AST> children = new ArrayList<>();
+            for (int i = 0; i < ast.children.size(); i++) {
+              children.add(ast.children.get(i));
+            }
+            childListStack.add(children);
+          }
+        }
+      }
+      return builder.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Token.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Token.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Token.java
index f2c56a5..4c94be0 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Token.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Token.java
@@ -34,7 +34,7 @@ public class Token<T> {
 
   @Override
   public String toString() {
-    return value.toString();
+    return "" + value;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 04c3954..db28adb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -71,6 +71,7 @@
         <base_hbase_version>1.1.1</base_hbase_version>
         <base_flume_version>1.5.2</base_flume_version>
         <!-- full dependency versions -->
+        <global_antlr_version>4.5</global_antlr_version>
         <global_opencsv_version>3.7</global_opencsv_version>
         <global_curator_version>2.7.1</global_curator_version>
         <global_storm_version>${base_storm_version}</global_storm_version>



[2/3] incubator-metron git commit: METRON-690: Create a DSL-based timestamp lookup for profiler to enable sparse windows closes apache/incubator-metron#450

Posted by ce...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowLexer.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowLexer.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowLexer.java
new file mode 100644
index 0000000..c6ce964
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowLexer.java
@@ -0,0 +1,477 @@
+// Generated from org/apache/metron/profiler/client/window/generated/Window.g4 by ANTLR 4.5
+package org.apache.metron.profiler.client.window.generated;
+
+//CHECKSTYLE:OFF
+/*
+ * 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.
+ */
+
+import org.antlr.v4.runtime.Lexer;
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.Token;
+import org.antlr.v4.runtime.TokenStream;
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.atn.*;
+import org.antlr.v4.runtime.dfa.DFA;
+import org.antlr.v4.runtime.misc.*;
+
+@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
+public class WindowLexer extends Lexer {
+	static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+
+	protected static final DFA[] _decisionToDFA;
+	protected static final PredictionContextCache _sharedContextCache =
+		new PredictionContextCache();
+	public static final int
+		COMMA=1, COLON=2, WINDOW=3, INCLUDE=4, EXCLUDE=5, NOW=6, FROM=7, EVERY=8, 
+		TO=9, AGO=10, NUMBER=11, IDENTIFIER=12, DAY_SPECIFIER=13, TIME_UNIT=14, 
+		WS=15;
+	public static String[] modeNames = {
+		"DEFAULT_MODE"
+	};
+
+	public static final String[] ruleNames = {
+		"COMMA", "COLON", "WINDOW", "INCLUDE", "EXCLUDE", "NOW", "FROM", "EVERY", 
+		"TO", "AGO", "NUMBER", "IDENTIFIER", "DAY_SPECIFIER", "TIME_UNIT", "WS", 
+		"SECOND_UNIT", "MINUTE_UNIT", "HOUR_UNIT", "DAY_UNIT", "MONDAY", "TUESDAY", 
+		"WEDNESDAY", "THURSDAY", "FRIDAY", "SATURDAY", "SUNDAY", "CURRENT_DAY_OF_WEEK", 
+		"WEEKEND", "WEEKDAY", "HOLIDAYS", "DATE", "DIGIT", "FIRST_DIGIT"
+	};
+
+	private static final String[] _LITERAL_NAMES = {
+		null, "','", "':'"
+	};
+	private static final String[] _SYMBOLIC_NAMES = {
+		null, "COMMA", "COLON", "WINDOW", "INCLUDE", "EXCLUDE", "NOW", "FROM", 
+		"EVERY", "TO", "AGO", "NUMBER", "IDENTIFIER", "DAY_SPECIFIER", "TIME_UNIT", 
+		"WS"
+	};
+	public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES);
+
+	/**
+	 * @deprecated Use {@link #VOCABULARY} instead.
+	 */
+	@Deprecated
+	public static final String[] tokenNames;
+	static {
+		tokenNames = new String[_SYMBOLIC_NAMES.length];
+		for (int i = 0; i < tokenNames.length; i++) {
+			tokenNames[i] = VOCABULARY.getLiteralName(i);
+			if (tokenNames[i] == null) {
+				tokenNames[i] = VOCABULARY.getSymbolicName(i);
+			}
+
+			if (tokenNames[i] == null) {
+				tokenNames[i] = "<INVALID>";
+			}
+		}
+	}
+
+	@Override
+	@Deprecated
+	public String[] getTokenNames() {
+		return tokenNames;
+	}
+
+	@Override
+
+	public Vocabulary getVocabulary() {
+		return VOCABULARY;
+	}
+
+
+	public WindowLexer(CharStream input) {
+		super(input);
+		_interp = new LexerATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache);
+	}
+
+	@Override
+	public String getGrammarFileName() { return "Window.g4"; }
+
+	@Override
+	public String[] getRuleNames() { return ruleNames; }
+
+	@Override
+	public String getSerializedATN() { return _serializedATN; }
+
+	@Override
+	public String[] getModeNames() { return modeNames; }
+
+	@Override
+	public ATN getATN() { return _ATN; }
+
+	public static final String _serializedATN =
+		"\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\2\21\u03d2\b\1\4\2"+
+		"\t\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4"+
+		"\13\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22"+
+		"\t\22\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31"+
+		"\t\31\4\32\t\32\4\33\t\33\4\34\t\34\4\35\t\35\4\36\t\36\4\37\t\37\4 \t"+
+		" \4!\t!\4\"\t\"\3\2\3\2\3\3\3\3\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3\4\3"+
+		"\4\3\4\3\4\3\4\5\4W\n\4\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3"+
+		"\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5"+
+		"\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3\5\3"+
+		"\5\3\5\5\5\u0089\n\5\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3"+
+		"\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6"+
+		"\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3\6\3"+
+		"\6\5\6\u00bb\n\6\3\7\3\7\3\7\3\7\3\7\3\7\5\7\u00c3\n\7\3\b\3\b\3\b\3\b"+
+		"\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3"+
+		"\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\3\b\5\b\u00e7\n\b\3\t\3"+
+		"\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t"+
+		"\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\3\t\5\t\u0105\n\t\3\n\3\n\3\n\3\n\3\n"+
+		"\3\n\3\n\3\n\3\n\3\n\3\n\3\n\3\n\3\n\5\n\u0115\n\n\3\13\3\13\3\13\3\13"+
+		"\3\13\3\13\5\13\u011d\n\13\3\f\3\f\7\f\u0121\n\f\f\f\16\f\u0124\13\f\3"+
+		"\r\3\r\3\r\7\r\u0129\n\r\f\r\16\r\u012c\13\r\3\16\3\16\3\16\3\16\3\16"+
+		"\3\16\3\16\3\16\3\16\3\16\3\16\3\16\5\16\u013a\n\16\3\17\3\17\3\17\3\17"+
+		"\5\17\u0140\n\17\3\20\6\20\u0143\n\20\r\20\16\20\u0144\3\20\3\20\3\21"+
+		"\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21"+
+		"\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21"+
+		"\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21\3\21"+
+		"\3\21\5\21\u0175\n\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22"+
+		"\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22"+
+		"\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22"+
+		"\3\22\3\22\3\22\3\22\3\22\3\22\5\22\u01a3\n\22\3\23\3\23\3\23\3\23\3\23"+
+		"\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23"+
+		"\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\3\23\5\23"+
+		"\u01c5\n\23\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24"+
+		"\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24\3\24"+
+		"\5\24\u01e1\n\24\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25"+
+		"\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25\3\25"+
+		"\3\25\5\25\u01fd\n\25\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26"+
+		"\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26\3\26"+
+		"\3\26\3\26\3\26\3\26\3\26\3\26\5\26\u021d\n\26\3\27\3\27\3\27\3\27\3\27"+
+		"\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27"+
+		"\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27\3\27"+
+		"\3\27\3\27\3\27\3\27\3\27\5\27\u0245\n\27\3\30\3\30\3\30\3\30\3\30\3\30"+
+		"\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30"+
+		"\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30\3\30"+
+		"\5\30\u0269\n\30\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31"+
+		"\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31\3\31"+
+		"\3\31\5\31\u0285\n\31\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32"+
+		"\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32"+
+		"\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\3\32\5\32\u02a9\n\32\3\33"+
+		"\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33"+
+		"\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\3\33\5\33\u02c5\n\33"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34\3\34"+
+		"\3\34\3\34\5\34\u0363\n\34\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35"+
+		"\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\35"+
+		"\3\35\3\35\3\35\3\35\3\35\3\35\3\35\5\35\u0383\n\35\3\36\3\36\3\36\3\36"+
+		"\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36"+
+		"\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\3\36\5\36\u03a3"+
+		"\n\36\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37"+
+		"\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37"+
+		"\3\37\3\37\3\37\5\37\u03c3\n\37\3 \3 \3 \3 \3 \3 \3 \3 \5 \u03cd\n \3"+
+		"!\3!\3\"\3\"\2\2#\3\3\5\4\7\5\t\6\13\7\r\b\17\t\21\n\23\13\25\f\27\r\31"+
+		"\16\33\17\35\20\37\21!\2#\2%\2\'\2)\2+\2-\2/\2\61\2\63\2\65\2\67\29\2"+
+		";\2=\2?\2A\2C\2\3\2\6\3\2<<\5\2\62;C\\c|\7\2/;C\\^^aac|\5\2\13\f\16\17"+
+		"\"\"\u0420\2\3\3\2\2\2\2\5\3\2\2\2\2\7\3\2\2\2\2\t\3\2\2\2\2\13\3\2\2"+
+		"\2\2\r\3\2\2\2\2\17\3\2\2\2\2\21\3\2\2\2\2\23\3\2\2\2\2\25\3\2\2\2\2\27"+
+		"\3\2\2\2\2\31\3\2\2\2\2\33\3\2\2\2\2\35\3\2\2\2\2\37\3\2\2\2\3E\3\2\2"+
+		"\2\5G\3\2\2\2\7V\3\2\2\2\t\u0088\3\2\2\2\13\u00ba\3\2\2\2\r\u00c2\3\2"+
+		"\2\2\17\u00e6\3\2\2\2\21\u0104\3\2\2\2\23\u0114\3\2\2\2\25\u011c\3\2\2"+
+		"\2\27\u011e\3\2\2\2\31\u0125\3\2\2\2\33\u0139\3\2\2\2\35\u013f\3\2\2\2"+
+		"\37\u0142\3\2\2\2!\u0174\3\2\2\2#\u01a2\3\2\2\2%\u01c4\3\2\2\2\'\u01e0"+
+		"\3\2\2\2)\u01fc\3\2\2\2+\u021c\3\2\2\2-\u0244\3\2\2\2/\u0268\3\2\2\2\61"+
+		"\u0284\3\2\2\2\63\u02a8\3\2\2\2\65\u02c4\3\2\2\2\67\u0362\3\2\2\29\u0382"+
+		"\3\2\2\2;\u03a2\3\2\2\2=\u03c2\3\2\2\2?\u03cc\3\2\2\2A\u03ce\3\2\2\2C"+
+		"\u03d0\3\2\2\2EF\7.\2\2F\4\3\2\2\2GH\7<\2\2H\6\3\2\2\2IJ\7y\2\2JK\7k\2"+
+		"\2KL\7p\2\2LM\7f\2\2MN\7q\2\2NW\7y\2\2OP\7y\2\2PQ\7k\2\2QR\7p\2\2RS\7"+
+		"f\2\2ST\7q\2\2TU\7y\2\2UW\7u\2\2VI\3\2\2\2VO\3\2\2\2W\b\3\2\2\2XY\7k\2"+
+		"\2YZ\7p\2\2Z[\7e\2\2[\\\7n\2\2\\]\7w\2\2]^\7f\2\2^\u0089\7g\2\2_`\7K\2"+
+		"\2`a\7P\2\2ab\7E\2\2bc\7N\2\2cd\7W\2\2de\7F\2\2e\u0089\7G\2\2fg\7k\2\2"+
+		"gh\7p\2\2hi\7e\2\2ij\7n\2\2jk\7w\2\2kl\7f\2\2lm\7g\2\2m\u0089\7u\2\2n"+
+		"o\7K\2\2op\7P\2\2pq\7E\2\2qr\7N\2\2rs\7W\2\2st\7F\2\2tu\7G\2\2u\u0089"+
+		"\7U\2\2vw\7k\2\2wx\7p\2\2xy\7e\2\2yz\7n\2\2z{\7w\2\2{|\7f\2\2|}\7k\2\2"+
+		"}~\7p\2\2~\u0089\7i\2\2\177\u0080\7K\2\2\u0080\u0081\7P\2\2\u0081\u0082"+
+		"\7E\2\2\u0082\u0083\7N\2\2\u0083\u0084\7W\2\2\u0084\u0085\7F\2\2\u0085"+
+		"\u0086\7K\2\2\u0086\u0087\7P\2\2\u0087\u0089\7I\2\2\u0088X\3\2\2\2\u0088"+
+		"_\3\2\2\2\u0088f\3\2\2\2\u0088n\3\2\2\2\u0088v\3\2\2\2\u0088\177\3\2\2"+
+		"\2\u0089\n\3\2\2\2\u008a\u008b\7g\2\2\u008b\u008c\7z\2\2\u008c\u008d\7"+
+		"e\2\2\u008d\u008e\7n\2\2\u008e\u008f\7w\2\2\u008f\u0090\7f\2\2\u0090\u00bb"+
+		"\7g\2\2\u0091\u0092\7G\2\2\u0092\u0093\7Z\2\2\u0093\u0094\7E\2\2\u0094"+
+		"\u0095\7N\2\2\u0095\u0096\7W\2\2\u0096\u0097\7F\2\2\u0097\u00bb\7G\2\2"+
+		"\u0098\u0099\7g\2\2\u0099\u009a\7z\2\2\u009a\u009b\7e\2\2\u009b\u009c"+
+		"\7n\2\2\u009c\u009d\7w\2\2\u009d\u009e\7f\2\2\u009e\u009f\7g\2\2\u009f"+
+		"\u00bb\7u\2\2\u00a0\u00a1\7G\2\2\u00a1\u00a2\7Z\2\2\u00a2\u00a3\7E\2\2"+
+		"\u00a3\u00a4\7N\2\2\u00a4\u00a5\7W\2\2\u00a5\u00a6\7F\2\2\u00a6\u00a7"+
+		"\7G\2\2\u00a7\u00bb\7U\2\2\u00a8\u00a9\7g\2\2\u00a9\u00aa\7z\2\2\u00aa"+
+		"\u00ab\7e\2\2\u00ab\u00ac\7n\2\2\u00ac\u00ad\7w\2\2\u00ad\u00ae\7f\2\2"+
+		"\u00ae\u00af\7k\2\2\u00af\u00b0\7p\2\2\u00b0\u00bb\7i\2\2\u00b1\u00b2"+
+		"\7G\2\2\u00b2\u00b3\7Z\2\2\u00b3\u00b4\7E\2\2\u00b4\u00b5\7N\2\2\u00b5"+
+		"\u00b6\7W\2\2\u00b6\u00b7\7F\2\2\u00b7\u00b8\7K\2\2\u00b8\u00b9\7P\2\2"+
+		"\u00b9\u00bb\7I\2\2\u00ba\u008a\3\2\2\2\u00ba\u0091\3\2\2\2\u00ba\u0098"+
+		"\3\2\2\2\u00ba\u00a0\3\2\2\2\u00ba\u00a8\3\2\2\2\u00ba\u00b1\3\2\2\2\u00bb"+
+		"\f\3\2\2\2\u00bc\u00bd\7P\2\2\u00bd\u00be\7Q\2\2\u00be\u00c3\7Y\2\2\u00bf"+
+		"\u00c0\7p\2\2\u00c0\u00c1\7q\2\2\u00c1\u00c3\7y\2\2\u00c2\u00bc\3\2\2"+
+		"\2\u00c2\u00bf\3\2\2\2\u00c3\16\3\2\2\2\u00c4\u00c5\7H\2\2\u00c5\u00c6"+
+		"\7T\2\2\u00c6\u00c7\7Q\2\2\u00c7\u00e7\7O\2\2\u00c8\u00c9\7h\2\2\u00c9"+
+		"\u00ca\7t\2\2\u00ca\u00cb\7q\2\2\u00cb\u00e7\7o\2\2\u00cc\u00cd\7U\2\2"+
+		"\u00cd\u00ce\7V\2\2\u00ce\u00cf\7C\2\2\u00cf\u00d0\7T\2\2\u00d0\u00d1"+
+		"\7V\2\2\u00d1\u00d2\7K\2\2\u00d2\u00d3\7P\2\2\u00d3\u00d4\7I\2\2\u00d4"+
+		"\u00d5\7\"\2\2\u00d5\u00d6\7H\2\2\u00d6\u00d7\7T\2\2\u00d7\u00d8\7Q\2"+
+		"\2\u00d8\u00e7\7O\2\2\u00d9\u00da\7u\2\2\u00da\u00db\7v\2\2\u00db\u00dc"+
+		"\7c\2\2\u00dc\u00dd\7t\2\2\u00dd\u00de\7v\2\2\u00de\u00df\7k\2\2\u00df"+
+		"\u00e0\7p\2\2\u00e0\u00e1\7i\2\2\u00e1\u00e2\7\"\2\2\u00e2\u00e3\7h\2"+
+		"\2\u00e3\u00e4\7t\2\2\u00e4\u00e5\7q\2\2\u00e5\u00e7\7o\2\2\u00e6\u00c4"+
+		"\3\2\2\2\u00e6\u00c8\3\2\2\2\u00e6\u00cc\3\2\2\2\u00e6\u00d9\3\2\2\2\u00e7"+
+		"\20\3\2\2\2\u00e8\u00e9\7G\2\2\u00e9\u00ea\7X\2\2\u00ea\u00eb\7G\2\2\u00eb"+
+		"\u00ec\7T\2\2\u00ec\u0105\7[\2\2\u00ed\u00ee\7g\2\2\u00ee\u00ef\7x\2\2"+
+		"\u00ef\u00f0\7g\2\2\u00f0\u00f1\7t\2\2\u00f1\u0105\7{\2\2\u00f2\u00f3"+
+		"\7H\2\2\u00f3\u00f4\7Q\2\2\u00f4\u00f5\7T\2\2\u00f5\u00f6\7\"\2\2\u00f6"+
+		"\u00f7\7G\2\2\u00f7\u00f8\7X\2\2\u00f8\u00f9\7G\2\2\u00f9\u00fa\7T\2\2"+
+		"\u00fa\u0105\7[\2\2\u00fb\u00fc\7h\2\2\u00fc\u00fd\7q\2\2\u00fd\u00fe"+
+		"\7t\2\2\u00fe\u00ff\7\"\2\2\u00ff\u0100\7g\2\2\u0100\u0101\7x\2\2\u0101"+
+		"\u0102\7g\2\2\u0102\u0103\7t\2\2\u0103\u0105\7{\2\2\u0104\u00e8\3\2\2"+
+		"\2\u0104\u00ed\3\2\2\2\u0104\u00f2\3\2\2\2\u0104\u00fb\3\2\2\2\u0105\22"+
+		"\3\2\2\2\u0106\u0107\7V\2\2\u0107\u0115\7Q\2\2\u0108\u0109\7v\2\2\u0109"+
+		"\u0115\7q\2\2\u010a\u010b\7w\2\2\u010b\u010c\7p\2\2\u010c\u010d\7v\2\2"+
+		"\u010d\u010e\7k\2\2\u010e\u0115\7n\2\2\u010f\u0110\7W\2\2\u0110\u0111"+
+		"\7P\2\2\u0111\u0112\7V\2\2\u0112\u0113\7K\2\2\u0113\u0115\7N\2\2\u0114"+
+		"\u0106\3\2\2\2\u0114\u0108\3\2\2\2\u0114\u010a\3\2\2\2\u0114\u010f\3\2"+
+		"\2\2\u0115\24\3\2\2\2\u0116\u0117\7C\2\2\u0117\u0118\7I\2\2\u0118\u011d"+
+		"\7Q\2\2\u0119\u011a\7c\2\2\u011a\u011b\7i\2\2\u011b\u011d\7q\2\2\u011c"+
+		"\u0116\3\2\2\2\u011c\u0119\3\2\2\2\u011d\26\3\2\2\2\u011e\u0122\5C\"\2"+
+		"\u011f\u0121\5A!\2\u0120\u011f\3\2\2\2\u0121\u0124\3\2\2\2\u0122\u0120"+
+		"\3\2\2\2\u0122\u0123\3\2\2\2\u0123\30\3\2\2\2\u0124\u0122\3\2\2\2\u0125"+
+		"\u0126\t\2\2\2\u0126\u012a\t\3\2\2\u0127\u0129\t\4\2\2\u0128\u0127\3\2"+
+		"\2\2\u0129\u012c\3\2\2\2\u012a\u0128\3\2\2\2\u012a\u012b\3\2\2\2\u012b"+
+		"\32\3\2\2\2\u012c\u012a\3\2\2\2\u012d\u013a\5)\25\2\u012e\u013a\5+\26"+
+		"\2\u012f\u013a\5-\27\2\u0130\u013a\5/\30\2\u0131\u013a\5\61\31\2\u0132"+
+		"\u013a\5\63\32\2\u0133\u013a\5\65\33\2\u0134\u013a\5\67\34\2\u0135\u013a"+
+		"\59\35\2\u0136\u013a\5;\36\2\u0137\u013a\5=\37\2\u0138\u013a\5? \2\u0139"+
+		"\u012d\3\2\2\2\u0139\u012e\3\2\2\2\u0139\u012f\3\2\2\2\u0139\u0130\3\2"+
+		"\2\2\u0139\u0131\3\2\2\2\u0139\u0132\3\2\2\2\u0139\u0133\3\2\2\2\u0139"+
+		"\u0134\3\2\2\2\u0139\u0135\3\2\2\2\u0139\u0136\3\2\2\2\u0139\u0137\3\2"+
+		"\2\2\u0139\u0138\3\2\2\2\u013a\34\3\2\2\2\u013b\u0140\5!\21\2\u013c\u0140"+
+		"\5#\22\2\u013d\u0140\5%\23\2\u013e\u0140\5\'\24\2\u013f\u013b\3\2\2\2"+
+		"\u013f\u013c\3\2\2\2\u013f\u013d\3\2\2\2\u013f\u013e\3\2\2\2\u0140\36"+
+		"\3\2\2\2\u0141\u0143\t\5\2\2\u0142\u0141\3\2\2\2\u0143\u0144\3\2\2\2\u0144"+
+		"\u0142\3\2\2\2\u0144\u0145\3\2\2\2\u0145\u0146\3\2\2\2\u0146\u0147\b\20"+
+		"\2\2\u0147 \3\2\2\2\u0148\u0149\7U\2\2\u0149\u014a\7G\2\2\u014a\u014b"+
+		"\7E\2\2\u014b\u014c\7Q\2\2\u014c\u014d\7P\2\2\u014d\u0175\7F\2\2\u014e"+
+		"\u014f\7u\2\2\u014f\u0150\7g\2\2\u0150\u0151\7e\2\2\u0151\u0152\7q\2\2"+
+		"\u0152\u0153\7p\2\2\u0153\u0175\7f\2\2\u0154\u0155\7u\2\2\u0155\u0156"+
+		"\7g\2\2\u0156\u0157\7e\2\2\u0157\u0158\7q\2\2\u0158\u0159\7p\2\2\u0159"+
+		"\u015a\7f\2\2\u015a\u0175\7u\2\2\u015b\u015c\7U\2\2\u015c\u015d\7G\2\2"+
+		"\u015d\u015e\7E\2\2\u015e\u015f\7Q\2\2\u015f\u0160\7P\2\2\u0160\u0161"+
+		"\7F\2\2\u0161\u0175\7U\2\2\u0162\u0163\7u\2\2\u0163\u0164\7g\2\2\u0164"+
+		"\u0165\7e\2\2\u0165\u0166\7q\2\2\u0166\u0167\7p\2\2\u0167\u0168\7f\2\2"+
+		"\u0168\u0169\7*\2\2\u0169\u016a\7u\2\2\u016a\u0175\7+\2\2\u016b\u016c"+
+		"\7U\2\2\u016c\u016d\7G\2\2\u016d\u016e\7E\2\2\u016e\u016f\7Q\2\2\u016f"+
+		"\u0170\7P\2\2\u0170\u0171\7F\2\2\u0171\u0172\7*\2\2\u0172\u0173\7U\2\2"+
+		"\u0173\u0175\7+\2\2\u0174\u0148\3\2\2\2\u0174\u014e\3\2\2\2\u0174\u0154"+
+		"\3\2\2\2\u0174\u015b\3\2\2\2\u0174\u0162\3\2\2\2\u0174\u016b\3\2\2\2\u0175"+
+		"\"\3\2\2\2\u0176\u0177\7O\2\2\u0177\u0178\7K\2\2\u0178\u0179\7P\2\2\u0179"+
+		"\u017a\7W\2\2\u017a\u017b\7V\2\2\u017b\u01a3\7G\2\2\u017c\u017d\7o\2\2"+
+		"\u017d\u017e\7k\2\2\u017e\u017f\7p\2\2\u017f\u0180\7w\2\2\u0180\u0181"+
+		"\7v\2\2\u0181\u01a3\7g\2\2\u0182\u0183\7o\2\2\u0183\u0184\7k\2\2\u0184"+
+		"\u0185\7p\2\2\u0185\u0186\7w\2\2\u0186\u0187\7v\2\2\u0187\u0188\7g\2\2"+
+		"\u0188\u01a3\7u\2\2\u0189\u018a\7O\2\2\u018a\u018b\7K\2\2\u018b\u018c"+
+		"\7P\2\2\u018c\u018d\7W\2\2\u018d\u018e\7V\2\2\u018e\u018f\7G\2\2\u018f"+
+		"\u01a3\7U\2\2\u0190\u0191\7o\2\2\u0191\u0192\7k\2\2\u0192\u0193\7p\2\2"+
+		"\u0193\u0194\7w\2\2\u0194\u0195\7v\2\2\u0195\u0196\7g\2\2\u0196\u0197"+
+		"\7*\2\2\u0197\u0198\7u\2\2\u0198\u01a3\7+\2\2\u0199\u019a\7O\2\2\u019a"+
+		"\u019b\7K\2\2\u019b\u019c\7P\2\2\u019c\u019d\7W\2\2\u019d\u019e\7V\2\2"+
+		"\u019e\u019f\7G\2\2\u019f\u01a0\7*\2\2\u01a0\u01a1\7U\2\2\u01a1\u01a3"+
+		"\7+\2\2\u01a2\u0176\3\2\2\2\u01a2\u017c\3\2\2\2\u01a2\u0182\3\2\2\2\u01a2"+
+		"\u0189\3\2\2\2\u01a2\u0190\3\2\2\2\u01a2\u0199\3\2\2\2\u01a3$\3\2\2\2"+
+		"\u01a4\u01a5\7J\2\2\u01a5\u01a6\7Q\2\2\u01a6\u01a7\7W\2\2\u01a7\u01c5"+
+		"\7T\2\2\u01a8\u01a9\7j\2\2\u01a9\u01aa\7q\2\2\u01aa\u01ab\7w\2\2\u01ab"+
+		"\u01c5\7t\2\2\u01ac\u01ad\7j\2\2\u01ad\u01ae\7q\2\2\u01ae\u01af\7w\2\2"+
+		"\u01af\u01b0\7t\2\2\u01b0\u01c5\7u\2\2\u01b1\u01b2\7J\2\2\u01b2\u01b3"+
+		"\7Q\2\2\u01b3\u01b4\7W\2\2\u01b4\u01b5\7T\2\2\u01b5\u01c5\7U\2\2\u01b6"+
+		"\u01b7\7j\2\2\u01b7\u01b8\7q\2\2\u01b8\u01b9\7w\2\2\u01b9\u01ba\7t\2\2"+
+		"\u01ba\u01bb\7*\2\2\u01bb\u01bc\7u\2\2\u01bc\u01c5\7+\2\2\u01bd\u01be"+
+		"\7J\2\2\u01be\u01bf\7Q\2\2\u01bf\u01c0\7W\2\2\u01c0\u01c1\7T\2\2\u01c1"+
+		"\u01c2\7*\2\2\u01c2\u01c3\7U\2\2\u01c3\u01c5\7+\2\2\u01c4\u01a4\3\2\2"+
+		"\2\u01c4\u01a8\3\2\2\2\u01c4\u01ac\3\2\2\2\u01c4\u01b1\3\2\2\2\u01c4\u01b6"+
+		"\3\2\2\2\u01c4\u01bd\3\2\2\2\u01c5&\3\2\2\2\u01c6\u01c7\7F\2\2\u01c7\u01c8"+
+		"\7C\2\2\u01c8\u01e1\7[\2\2\u01c9\u01ca\7f\2\2\u01ca\u01cb\7c\2\2\u01cb"+
+		"\u01e1\7{\2\2\u01cc\u01cd\7f\2\2\u01cd\u01ce\7c\2\2\u01ce\u01cf\7{\2\2"+
+		"\u01cf\u01e1\7u\2\2\u01d0\u01d1\7F\2\2\u01d1\u01d2\7C\2\2\u01d2\u01d3"+
+		"\7[\2\2\u01d3\u01e1\7U\2\2\u01d4\u01d5\7f\2\2\u01d5\u01d6\7c\2\2\u01d6"+
+		"\u01d7\7{\2\2\u01d7\u01d8\7*\2\2\u01d8\u01d9\7u\2\2\u01d9\u01e1\7+\2\2"+
+		"\u01da\u01db\7F\2\2\u01db\u01dc\7C\2\2\u01dc\u01dd\7[\2\2\u01dd\u01de"+
+		"\7*\2\2\u01de\u01df\7U\2\2\u01df\u01e1\7+\2\2\u01e0\u01c6\3\2\2\2\u01e0"+
+		"\u01c9\3\2\2\2\u01e0\u01cc\3\2\2\2\u01e0\u01d0\3\2\2\2\u01e0\u01d4\3\2"+
+		"\2\2\u01e0\u01da\3\2\2\2\u01e1(\3\2\2\2\u01e2\u01e3\7O\2\2\u01e3\u01e4"+
+		"\7Q\2\2\u01e4\u01e5\7P\2\2\u01e5\u01e6\7F\2\2\u01e6\u01e7\7C\2\2\u01e7"+
+		"\u01fd\7[\2\2\u01e8\u01e9\7o\2\2\u01e9\u01ea\7q\2\2\u01ea\u01eb\7p\2\2"+
+		"\u01eb\u01ec\7f\2\2\u01ec\u01ed\7c\2\2\u01ed\u01fd\7{\2\2\u01ee\u01ef"+
+		"\7O\2\2\u01ef\u01f0\7Q\2\2\u01f0\u01f1\7P\2\2\u01f1\u01f2\7F\2\2\u01f2"+
+		"\u01f3\7C\2\2\u01f3\u01f4\7[\2\2\u01f4\u01fd\7U\2\2\u01f5\u01f6\7o\2\2"+
+		"\u01f6\u01f7\7q\2\2\u01f7\u01f8\7p\2\2\u01f8\u01f9\7f\2\2\u01f9\u01fa"+
+		"\7c\2\2\u01fa\u01fb\7{\2\2\u01fb\u01fd\7u\2\2\u01fc\u01e2\3\2\2\2\u01fc"+
+		"\u01e8\3\2\2\2\u01fc\u01ee\3\2\2\2\u01fc\u01f5\3\2\2\2\u01fd*\3\2\2\2"+
+		"\u01fe\u01ff\7V\2\2\u01ff\u0200\7W\2\2\u0200\u0201\7G\2\2\u0201\u0202"+
+		"\7U\2\2\u0202\u0203\7F\2\2\u0203\u0204\7C\2\2\u0204\u021d\7[\2\2\u0205"+
+		"\u0206\7v\2\2\u0206\u0207\7w\2\2\u0207\u0208\7g\2\2\u0208\u0209\7u\2\2"+
+		"\u0209\u020a\7f\2\2\u020a\u020b\7c\2\2\u020b\u021d\7{\2\2\u020c\u020d"+
+		"\7V\2\2\u020d\u020e\7W\2\2\u020e\u020f\7G\2\2\u020f\u0210\7U\2\2\u0210"+
+		"\u0211\7F\2\2\u0211\u0212\7C\2\2\u0212\u0213\7[\2\2\u0213\u021d\7U\2\2"+
+		"\u0214\u0215\7v\2\2\u0215\u0216\7w\2\2\u0216\u0217\7g\2\2\u0217\u0218"+
+		"\7u\2\2\u0218\u0219\7f\2\2\u0219\u021a\7c\2\2\u021a\u021b\7{\2\2\u021b"+
+		"\u021d\7u\2\2\u021c\u01fe\3\2\2\2\u021c\u0205\3\2\2\2\u021c\u020c\3\2"+
+		"\2\2\u021c\u0214\3\2\2\2\u021d,\3\2\2\2\u021e\u021f\7Y\2\2\u021f\u0220"+
+		"\7G\2\2\u0220\u0221\7F\2\2\u0221\u0222\7P\2\2\u0222\u0223\7G\2\2\u0223"+
+		"\u0224\7U\2\2\u0224\u0225\7F\2\2\u0225\u0226\7C\2\2\u0226\u0245\7[\2\2"+
+		"\u0227\u0228\7y\2\2\u0228\u0229\7g\2\2\u0229\u022a\7f\2\2\u022a\u022b"+
+		"\7p\2\2\u022b\u022c\7g\2\2\u022c\u022d\7u\2\2\u022d\u022e\7f\2\2\u022e"+
+		"\u022f\7c\2\2\u022f\u0245\7{\2\2\u0230\u0231\7Y\2\2\u0231\u0232\7G\2\2"+
+		"\u0232\u0233\7F\2\2\u0233\u0234\7P\2\2\u0234\u0235\7G\2\2\u0235\u0236"+
+		"\7U\2\2\u0236\u0237\7F\2\2\u0237\u0238\7C\2\2\u0238\u0239\7[\2\2\u0239"+
+		"\u0245\7U\2\2\u023a\u023b\7y\2\2\u023b\u023c\7g\2\2\u023c\u023d\7f\2\2"+
+		"\u023d\u023e\7p\2\2\u023e\u023f\7g\2\2\u023f\u0240\7u\2\2\u0240\u0241"+
+		"\7f\2\2\u0241\u0242\7c\2\2\u0242\u0243\7{\2\2\u0243\u0245\7u\2\2\u0244"+
+		"\u021e\3\2\2\2\u0244\u0227\3\2\2\2\u0244\u0230\3\2\2\2\u0244\u023a\3\2"+
+		"\2\2\u0245.\3\2\2\2\u0246\u0247\7V\2\2\u0247\u0248\7J\2\2\u0248\u0249"+
+		"\7W\2\2\u0249\u024a\7T\2\2\u024a\u024b\7U\2\2\u024b\u024c\7F\2\2\u024c"+
+		"\u024d\7C\2\2\u024d\u0269\7[\2\2\u024e\u024f\7v\2\2\u024f\u0250\7j\2\2"+
+		"\u0250\u0251\7w\2\2\u0251\u0252\7t\2\2\u0252\u0253\7u\2\2\u0253\u0254"+
+		"\7f\2\2\u0254\u0255\7c\2\2\u0255\u0269\7{\2\2\u0256\u0257\7V\2\2\u0257"+
+		"\u0258\7J\2\2\u0258\u0259\7W\2\2\u0259\u025a\7T\2\2\u025a\u025b\7U\2\2"+
+		"\u025b\u025c\7F\2\2\u025c\u025d\7C\2\2\u025d\u025e\7[\2\2\u025e\u0269"+
+		"\7U\2\2\u025f\u0260\7v\2\2\u0260\u0261\7j\2\2\u0261\u0262\7w\2\2\u0262"+
+		"\u0263\7t\2\2\u0263\u0264\7u\2\2\u0264\u0265\7f\2\2\u0265\u0266\7c\2\2"+
+		"\u0266\u0267\7{\2\2\u0267\u0269\7u\2\2\u0268\u0246\3\2\2\2\u0268\u024e"+
+		"\3\2\2\2\u0268\u0256\3\2\2\2\u0268\u025f\3\2\2\2\u0269\60\3\2\2\2\u026a"+
+		"\u026b\7H\2\2\u026b\u026c\7T\2\2\u026c\u026d\7K\2\2\u026d\u026e\7F\2\2"+
+		"\u026e\u026f\7C\2\2\u026f\u0285\7[\2\2\u0270\u0271\7h\2\2\u0271\u0272"+
+		"\7t\2\2\u0272\u0273\7k\2\2\u0273\u0274\7f\2\2\u0274\u0275\7c\2\2\u0275"+
+		"\u0285\7{\2\2\u0276\u0277\7H\2\2\u0277\u0278\7T\2\2\u0278\u0279\7K\2\2"+
+		"\u0279\u027a\7F\2\2\u027a\u027b\7C\2\2\u027b\u027c\7[\2\2\u027c\u0285"+
+		"\7U\2\2\u027d\u027e\7h\2\2\u027e\u027f\7t\2\2\u027f\u0280\7k\2\2\u0280"+
+		"\u0281\7f\2\2\u0281\u0282\7c\2\2\u0282\u0283\7{\2\2\u0283\u0285\7u\2\2"+
+		"\u0284\u026a\3\2\2\2\u0284\u0270\3\2\2\2\u0284\u0276\3\2\2\2\u0284\u027d"+
+		"\3\2\2\2\u0285\62\3\2\2\2\u0286\u0287\7U\2\2\u0287\u0288\7C\2\2\u0288"+
+		"\u0289\7V\2\2\u0289\u028a\7W\2\2\u028a\u028b\7T\2\2\u028b\u028c\7F\2\2"+
+		"\u028c\u028d\7C\2\2\u028d\u02a9\7[\2\2\u028e\u028f\7u\2\2\u028f\u0290"+
+		"\7c\2\2\u0290\u0291\7v\2\2\u0291\u0292\7w\2\2\u0292\u0293\7t\2\2\u0293"+
+		"\u0294\7f\2\2\u0294\u0295\7c\2\2\u0295\u02a9\7{\2\2\u0296\u0297\7U\2\2"+
+		"\u0297\u0298\7C\2\2\u0298\u0299\7V\2\2\u0299\u029a\7W\2\2\u029a\u029b"+
+		"\7T\2\2\u029b\u029c\7F\2\2\u029c\u029d\7C\2\2\u029d\u029e\7[\2\2\u029e"+
+		"\u02a9\7U\2\2\u029f\u02a0\7u\2\2\u02a0\u02a1\7c\2\2\u02a1\u02a2\7v\2\2"+
+		"\u02a2\u02a3\7w\2\2\u02a3\u02a4\7t\2\2\u02a4\u02a5\7f\2\2\u02a5\u02a6"+
+		"\7c\2\2\u02a6\u02a7\7{\2\2\u02a7\u02a9\7u\2\2\u02a8\u0286\3\2\2\2\u02a8"+
+		"\u028e\3\2\2\2\u02a8\u0296\3\2\2\2\u02a8\u029f\3\2\2\2\u02a9\64\3\2\2"+
+		"\2\u02aa\u02ab\7U\2\2\u02ab\u02ac\7W\2\2\u02ac\u02ad\7P\2\2\u02ad\u02ae"+
+		"\7F\2\2\u02ae\u02af\7C\2\2\u02af\u02c5\7[\2\2\u02b0\u02b1\7u\2\2\u02b1"+
+		"\u02b2\7w\2\2\u02b2\u02b3\7p\2\2\u02b3\u02b4\7f\2\2\u02b4\u02b5\7c\2\2"+
+		"\u02b5\u02c5\7{\2\2\u02b6\u02b7\7U\2\2\u02b7\u02b8\7W\2\2\u02b8\u02b9"+
+		"\7P\2\2\u02b9\u02ba\7F\2\2\u02ba\u02bb\7C\2\2\u02bb\u02bc\7[\2\2\u02bc"+
+		"\u02c5\7U\2\2\u02bd\u02be\7u\2\2\u02be\u02bf\7w\2\2\u02bf\u02c0\7p\2\2"+
+		"\u02c0\u02c1\7f\2\2\u02c1\u02c2\7c\2\2\u02c2\u02c3\7{\2\2\u02c3\u02c5"+
+		"\7u\2\2\u02c4\u02aa\3\2\2\2\u02c4\u02b0\3\2\2\2\u02c4\u02b6\3\2\2\2\u02c4"+
+		"\u02bd\3\2\2\2\u02c5\66\3\2\2\2\u02c6\u02c7\7v\2\2\u02c7\u02c8\7j\2\2"+
+		"\u02c8\u02c9\7k\2\2\u02c9\u02ca\7u\2\2\u02ca\u02cb\7\"\2\2\u02cb\u02cc"+
+		"\7f\2\2\u02cc\u02cd\7c\2\2\u02cd\u02ce\7{\2\2\u02ce\u02cf\7\"\2\2\u02cf"+
+		"\u02d0\7q\2\2\u02d0\u02d1\7h\2\2\u02d1\u02d2\7\"\2\2\u02d2\u02d3\7y\2"+
+		"\2\u02d3\u02d4\7g\2\2\u02d4\u02d5\7g\2\2\u02d5\u0363\7m\2\2\u02d6\u02d7"+
+		"\7V\2\2\u02d7\u02d8\7J\2\2\u02d8\u02d9\7K\2\2\u02d9\u02da\7U\2\2\u02da"+
+		"\u02db\7\"\2\2\u02db\u02dc\7F\2\2\u02dc\u02dd\7C\2\2\u02dd\u02de\7[\2"+
+		"\2\u02de\u02df\7\"\2\2\u02df\u02e0\7Q\2\2\u02e0\u02e1\7H\2\2\u02e1\u02e2"+
+		"\7\"\2\2\u02e2\u02e3\7Y\2\2\u02e3\u02e4\7G\2\2\u02e4\u02e5\7G\2\2\u02e5"+
+		"\u0363\7M\2\2\u02e6\u02e7\7v\2\2\u02e7\u02e8\7j\2\2\u02e8\u02e9\7k\2\2"+
+		"\u02e9\u02ea\7u\2\2\u02ea\u02eb\7\"\2\2\u02eb\u02ec\7f\2\2\u02ec\u02ed"+
+		"\7c\2\2\u02ed\u02ee\7{\2\2\u02ee\u02ef\7\"\2\2\u02ef\u02f0\7q\2\2\u02f0"+
+		"\u02f1\7h\2\2\u02f1\u02f2\7\"\2\2\u02f2\u02f3\7v\2\2\u02f3\u02f4\7j\2"+
+		"\2\u02f4\u02f5\7g\2\2\u02f5\u02f6\7\"\2\2\u02f6\u02f7\7y\2\2\u02f7\u02f8"+
+		"\7g\2\2\u02f8\u02f9\7g\2\2\u02f9\u0363\7m\2\2\u02fa\u02fb\7V\2\2\u02fb"+
+		"\u02fc\7J\2\2\u02fc\u02fd\7K\2\2\u02fd\u02fe\7U\2\2\u02fe\u02ff\7\"\2"+
+		"\2\u02ff\u0300\7F\2\2\u0300\u0301\7C\2\2\u0301\u0302\7[\2\2\u0302\u0303"+
+		"\7\"\2\2\u0303\u0304\7Q\2\2\u0304\u0305\7H\2\2\u0305\u0306\7\"\2\2\u0306"+
+		"\u0307\7V\2\2\u0307\u0308\7J\2\2\u0308\u0309\7G\2\2\u0309\u030a\7\"\2"+
+		"\2\u030a\u030b\7Y\2\2\u030b\u030c\7G\2\2\u030c\u030d\7G\2\2\u030d\u0363"+
+		"\7M\2\2\u030e\u030f\7e\2\2\u030f\u0310\7w\2\2\u0310\u0311\7t\2\2\u0311"+
+		"\u0312\7t\2\2\u0312\u0313\7g\2\2\u0313\u0314\7p\2\2\u0314\u0315\7v\2\2"+
+		"\u0315\u0316\7\"\2\2\u0316\u0317\7f\2\2\u0317\u0318\7c\2\2\u0318\u0319"+
+		"\7{\2\2\u0319\u031a\7\"\2\2\u031a\u031b\7q\2\2\u031b\u031c\7h\2\2\u031c"+
+		"\u031d\7\"\2\2\u031d\u031e\7y\2\2\u031e\u031f\7g\2\2\u031f\u0320\7g\2"+
+		"\2\u0320\u0363\7m\2\2\u0321\u0322\7E\2\2\u0322\u0323\7W\2\2\u0323\u0324"+
+		"\7T\2\2\u0324\u0325\7T\2\2\u0325\u0326\7G\2\2\u0326\u0327\7P\2\2\u0327"+
+		"\u0328\7V\2\2\u0328\u0329\7\"\2\2\u0329\u032a\7F\2\2\u032a\u032b\7C\2"+
+		"\2\u032b\u032c\7[\2\2\u032c\u032d\7\"\2\2\u032d\u032e\7Q\2\2\u032e\u032f"+
+		"\7H\2\2\u032f\u0330\7\"\2\2\u0330\u0331\7Y\2\2\u0331\u0332\7G\2\2\u0332"+
+		"\u0333\7G\2\2\u0333\u0363\7M\2\2\u0334\u0335\7e\2\2\u0335\u0336\7w\2\2"+
+		"\u0336\u0337\7t\2\2\u0337\u0338\7t\2\2\u0338\u0339\7g\2\2\u0339\u033a"+
+		"\7p\2\2\u033a\u033b\7v\2\2\u033b\u033c\7\"\2\2\u033c\u033d\7f\2\2\u033d"+
+		"\u033e\7c\2\2\u033e\u033f\7{\2\2\u033f\u0340\7\"\2\2\u0340\u0341\7q\2"+
+		"\2\u0341\u0342\7h\2\2\u0342\u0343\7\"\2\2\u0343\u0344\7v\2\2\u0344\u0345"+
+		"\7j\2\2\u0345\u0346\7g\2\2\u0346\u0347\7\"\2\2\u0347\u0348\7y\2\2\u0348"+
+		"\u0349\7g\2\2\u0349\u034a\7g\2\2\u034a\u0363\7m\2\2\u034b\u034c\7E\2\2"+
+		"\u034c\u034d\7W\2\2\u034d\u034e\7T\2\2\u034e\u034f\7T\2\2\u034f\u0350"+
+		"\7G\2\2\u0350\u0351\7P\2\2\u0351\u0352\7V\2\2\u0352\u0353\7\"\2\2\u0353"+
+		"\u0354\7F\2\2\u0354\u0355\7C\2\2\u0355\u0356\7[\2\2\u0356\u0357\7\"\2"+
+		"\2\u0357\u0358\7Q\2\2\u0358\u0359\7H\2\2\u0359\u035a\7\"\2\2\u035a\u035b"+
+		"\7V\2\2\u035b\u035c\7J\2\2\u035c\u035d\7G\2\2\u035d\u035e\7\"\2\2\u035e"+
+		"\u035f\7Y\2\2\u035f\u0360\7G\2\2\u0360\u0361\7G\2\2\u0361\u0363\7M\2\2"+
+		"\u0362\u02c6\3\2\2\2\u0362\u02d6\3\2\2\2\u0362\u02e6\3\2\2\2\u0362\u02fa"+
+		"\3\2\2\2\u0362\u030e\3\2\2\2\u0362\u0321\3\2\2\2\u0362\u0334\3\2\2\2\u0362"+
+		"\u034b\3\2\2\2\u03638\3\2\2\2\u0364\u0365\7y\2\2\u0365\u0366\7g\2\2\u0366"+
+		"\u0367\7g\2\2\u0367\u0368\7m\2\2\u0368\u0369\7g\2\2\u0369\u036a\7p\2\2"+
+		"\u036a\u0383\7f\2\2\u036b\u036c\7Y\2\2\u036c\u036d\7G\2\2\u036d\u036e"+
+		"\7G\2\2\u036e\u036f\7M\2\2\u036f\u0370\7G\2\2\u0370\u0371\7P\2\2\u0371"+
+		"\u0383\7F\2\2\u0372\u0373\7y\2\2\u0373\u0374\7g\2\2\u0374\u0375\7g\2\2"+
+		"\u0375\u0376\7m\2\2\u0376\u0377\7g\2\2\u0377\u0378\7p\2\2\u0378\u0379"+
+		"\7f\2\2\u0379\u0383\7u\2\2\u037a\u037b\7Y\2\2\u037b\u037c\7G\2\2\u037c"+
+		"\u037d\7G\2\2\u037d\u037e\7M\2\2\u037e\u037f\7G\2\2\u037f\u0380\7P\2\2"+
+		"\u0380\u0381\7F\2\2\u0381\u0383\7U\2\2\u0382\u0364\3\2\2\2\u0382\u036b"+
+		"\3\2\2\2\u0382\u0372\3\2\2\2\u0382\u037a\3\2\2\2\u0383:\3\2\2\2\u0384"+
+		"\u0385\7y\2\2\u0385\u0386\7g\2\2\u0386\u0387\7g\2\2\u0387\u0388\7m\2\2"+
+		"\u0388\u0389\7f\2\2\u0389\u038a\7c\2\2\u038a\u03a3\7{\2\2\u038b\u038c"+
+		"\7Y\2\2\u038c\u038d\7G\2\2\u038d\u038e\7G\2\2\u038e\u038f\7M\2\2\u038f"+
+		"\u0390\7F\2\2\u0390\u0391\7C\2\2\u0391\u03a3\7[\2\2\u0392\u0393\7y\2\2"+
+		"\u0393\u0394\7g\2\2\u0394\u0395\7g\2\2\u0395\u0396\7m\2\2\u0396\u0397"+
+		"\7f\2\2\u0397\u0398\7c\2\2\u0398\u0399\7{\2\2\u0399\u03a3\7u\2\2\u039a"+
+		"\u039b\7Y\2\2\u039b\u039c\7G\2\2\u039c\u039d\7G\2\2\u039d\u039e\7M\2\2"+
+		"\u039e\u039f\7F\2\2\u039f\u03a0\7C\2\2\u03a0\u03a1\7[\2\2\u03a1\u03a3"+
+		"\7U\2\2\u03a2\u0384\3\2\2\2\u03a2\u038b\3\2\2\2\u03a2\u0392\3\2\2\2\u03a2"+
+		"\u039a\3\2\2\2\u03a3<\3\2\2\2\u03a4\u03a5\7j\2\2\u03a5\u03a6\7q\2\2\u03a6"+
+		"\u03a7\7n\2\2\u03a7\u03a8\7k\2\2\u03a8\u03a9\7f\2\2\u03a9\u03aa\7c\2\2"+
+		"\u03aa\u03c3\7{\2\2\u03ab\u03ac\7J\2\2\u03ac\u03ad\7Q\2\2\u03ad\u03ae"+
+		"\7N\2\2\u03ae\u03af\7K\2\2\u03af\u03b0\7F\2\2\u03b0\u03b1\7C\2\2\u03b1"+
+		"\u03c3\7[\2\2\u03b2\u03b3\7j\2\2\u03b3\u03b4\7q\2\2\u03b4\u03b5\7n\2\2"+
+		"\u03b5\u03b6\7k\2\2\u03b6\u03b7\7f\2\2\u03b7\u03b8\7c\2\2\u03b8\u03b9"+
+		"\7{\2\2\u03b9\u03c3\7u\2\2\u03ba\u03bb\7J\2\2\u03bb\u03bc\7Q\2\2\u03bc"+
+		"\u03bd\7N\2\2\u03bd\u03be\7K\2\2\u03be\u03bf\7F\2\2\u03bf\u03c0\7C\2\2"+
+		"\u03c0\u03c1\7[\2\2\u03c1\u03c3\7U\2\2\u03c2\u03a4\3\2\2\2\u03c2\u03ab"+
+		"\3\2\2\2\u03c2\u03b2\3\2\2\2\u03c2\u03ba\3\2\2\2\u03c3>\3\2\2\2\u03c4"+
+		"\u03c5\7f\2\2\u03c5\u03c6\7c\2\2\u03c6\u03c7\7v\2\2\u03c7\u03cd\7g\2\2"+
+		"\u03c8\u03c9\7F\2\2\u03c9\u03ca\7C\2\2\u03ca\u03cb\7V\2\2\u03cb\u03cd"+
+		"\7G\2\2\u03cc\u03c4\3\2\2\2\u03cc\u03c8\3\2\2\2\u03cd@\3\2\2\2\u03ce\u03cf"+
+		"\4\62;\2\u03cfB\3\2\2\2\u03d0\u03d1\4\63;\2\u03d1D\3\2\2\2 \2V\u0088\u00ba"+
+		"\u00c2\u00e6\u0104\u0114\u011c\u0122\u012a\u0139\u013f\u0144\u0174\u01a2"+
+		"\u01c4\u01e0\u01fc\u021c\u0244\u0268\u0284\u02a8\u02c4\u0362\u0382\u03a2"+
+		"\u03c2\u03cc\3\b\2\2";
+	public static final ATN _ATN =
+		new ATNDeserializer().deserialize(_serializedATN.toCharArray());
+	static {
+		_decisionToDFA = new DFA[_ATN.getNumberOfDecisions()];
+		for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) {
+			_decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowListener.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowListener.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowListener.java
new file mode 100644
index 0000000..0cd8f5d
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowListener.java
@@ -0,0 +1,231 @@
+// Generated from org/apache/metron/profiler/client/window/generated/Window.g4 by ANTLR 4.5
+package org.apache.metron.profiler.client.window.generated;
+
+//CHECKSTYLE:OFF
+/*
+ * 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.
+ */
+
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.ParseTreeListener;
+
+/**
+ * This interface defines a complete listener for a process tree produced by
+ * {@link WindowParser}.
+ */
+public interface WindowListener extends ParseTreeListener {
+	/**
+	 * Enter a process tree produced by {@link WindowParser#window}.
+	 * @param ctx the process tree
+	 */
+	void enterWindow(WindowParser.WindowContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#window}.
+	 * @param ctx the process tree
+	 */
+	void exitWindow(WindowParser.WindowContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code NonRepeatingWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void enterNonRepeatingWindow(WindowParser.NonRepeatingWindowContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code NonRepeatingWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void exitNonRepeatingWindow(WindowParser.NonRepeatingWindowContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code RepeatingWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void enterRepeatingWindow(WindowParser.RepeatingWindowContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code RepeatingWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void exitRepeatingWindow(WindowParser.RepeatingWindowContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code DenseWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void enterDenseWindow(WindowParser.DenseWindowContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code DenseWindow}
+	 * labeled alternative in {@link WindowParser#window_expression}.
+	 * @param ctx the process tree
+	 */
+	void exitDenseWindow(WindowParser.DenseWindowContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#excluding_specifier}.
+	 * @param ctx the process tree
+	 */
+	void enterExcluding_specifier(WindowParser.Excluding_specifierContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#excluding_specifier}.
+	 * @param ctx the process tree
+	 */
+	void exitExcluding_specifier(WindowParser.Excluding_specifierContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#including_specifier}.
+	 * @param ctx the process tree
+	 */
+	void enterIncluding_specifier(WindowParser.Including_specifierContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#including_specifier}.
+	 * @param ctx the process tree
+	 */
+	void exitIncluding_specifier(WindowParser.Including_specifierContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#specifier}.
+	 * @param ctx the process tree
+	 */
+	void enterSpecifier(WindowParser.SpecifierContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#specifier}.
+	 * @param ctx the process tree
+	 */
+	void exitSpecifier(WindowParser.SpecifierContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#specifier_arg_list}.
+	 * @param ctx the process tree
+	 */
+	void enterSpecifier_arg_list(WindowParser.Specifier_arg_listContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#specifier_arg_list}.
+	 * @param ctx the process tree
+	 */
+	void exitSpecifier_arg_list(WindowParser.Specifier_arg_listContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#day_specifier}.
+	 * @param ctx the process tree
+	 */
+	void enterDay_specifier(WindowParser.Day_specifierContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#day_specifier}.
+	 * @param ctx the process tree
+	 */
+	void exitDay_specifier(WindowParser.Day_specifierContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#identifier}.
+	 * @param ctx the process tree
+	 */
+	void enterIdentifier(WindowParser.IdentifierContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#identifier}.
+	 * @param ctx the process tree
+	 */
+	void exitIdentifier(WindowParser.IdentifierContext ctx);
+	/**
+	 * Enter a process tree produced by {@link WindowParser#specifier_list}.
+	 * @param ctx the process tree
+	 */
+	void enterSpecifier_list(WindowParser.Specifier_listContext ctx);
+	/**
+	 * Exit a process tree produced by {@link WindowParser#specifier_list}.
+	 * @param ctx the process tree
+	 */
+	void exitSpecifier_list(WindowParser.Specifier_listContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code FromToDuration}
+	 * labeled alternative in {@link WindowParser#duration}.
+	 * @param ctx the process tree
+	 */
+	void enterFromToDuration(WindowParser.FromToDurationContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code FromToDuration}
+	 * labeled alternative in {@link WindowParser#duration}.
+	 * @param ctx the process tree
+	 */
+	void exitFromToDuration(WindowParser.FromToDurationContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code FromDuration}
+	 * labeled alternative in {@link WindowParser#duration}.
+	 * @param ctx the process tree
+	 */
+	void enterFromDuration(WindowParser.FromDurationContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code FromDuration}
+	 * labeled alternative in {@link WindowParser#duration}.
+	 * @param ctx the process tree
+	 */
+	void exitFromDuration(WindowParser.FromDurationContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code SkipDistance}
+	 * labeled alternative in {@link WindowParser#skip_distance}.
+	 * @param ctx the process tree
+	 */
+	void enterSkipDistance(WindowParser.SkipDistanceContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code SkipDistance}
+	 * labeled alternative in {@link WindowParser#skip_distance}.
+	 * @param ctx the process tree
+	 */
+	void exitSkipDistance(WindowParser.SkipDistanceContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code WindowWidth}
+	 * labeled alternative in {@link WindowParser#window_width}.
+	 * @param ctx the process tree
+	 */
+	void enterWindowWidth(WindowParser.WindowWidthContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code WindowWidth}
+	 * labeled alternative in {@link WindowParser#window_width}.
+	 * @param ctx the process tree
+	 */
+	void exitWindowWidth(WindowParser.WindowWidthContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code TimeInterval}
+	 * labeled alternative in {@link WindowParser#time_interval}.
+	 * @param ctx the process tree
+	 */
+	void enterTimeInterval(WindowParser.TimeIntervalContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code TimeInterval}
+	 * labeled alternative in {@link WindowParser#time_interval}.
+	 * @param ctx the process tree
+	 */
+	void exitTimeInterval(WindowParser.TimeIntervalContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code TimeAmount}
+	 * labeled alternative in {@link WindowParser#time_amount}.
+	 * @param ctx the process tree
+	 */
+	void enterTimeAmount(WindowParser.TimeAmountContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code TimeAmount}
+	 * labeled alternative in {@link WindowParser#time_amount}.
+	 * @param ctx the process tree
+	 */
+	void exitTimeAmount(WindowParser.TimeAmountContext ctx);
+	/**
+	 * Enter a process tree produced by the {@code TimeUnit}
+	 * labeled alternative in {@link WindowParser#time_unit}.
+	 * @param ctx the process tree
+	 */
+	void enterTimeUnit(WindowParser.TimeUnitContext ctx);
+	/**
+	 * Exit a process tree produced by the {@code TimeUnit}
+	 * labeled alternative in {@link WindowParser#time_unit}.
+	 * @param ctx the process tree
+	 */
+	void exitTimeUnit(WindowParser.TimeUnitContext ctx);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowParser.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowParser.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowParser.java
new file mode 100644
index 0000000..a762644
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowParser.java
@@ -0,0 +1,1096 @@
+// Generated from org/apache/metron/profiler/client/window/generated/Window.g4 by ANTLR 4.5
+package org.apache.metron.profiler.client.window.generated;
+
+//CHECKSTYLE:OFF
+/*
+ * 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.
+ */
+
+import org.antlr.v4.runtime.atn.*;
+import org.antlr.v4.runtime.dfa.DFA;
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.misc.*;
+import org.antlr.v4.runtime.tree.*;
+import java.util.List;
+import java.util.Iterator;
+import java.util.ArrayList;
+
+@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"})
+public class WindowParser extends Parser {
+	static { RuntimeMetaData.checkVersion("4.5", RuntimeMetaData.VERSION); }
+
+	protected static final DFA[] _decisionToDFA;
+	protected static final PredictionContextCache _sharedContextCache =
+		new PredictionContextCache();
+	public static final int
+		COMMA=1, COLON=2, WINDOW=3, INCLUDE=4, EXCLUDE=5, NOW=6, FROM=7, EVERY=8, 
+		TO=9, AGO=10, NUMBER=11, IDENTIFIER=12, DAY_SPECIFIER=13, TIME_UNIT=14, 
+		WS=15;
+	public static final int
+		RULE_window = 0, RULE_window_expression = 1, RULE_excluding_specifier = 2, 
+		RULE_including_specifier = 3, RULE_specifier = 4, RULE_specifier_arg_list = 5, 
+		RULE_day_specifier = 6, RULE_identifier = 7, RULE_specifier_list = 8, 
+		RULE_duration = 9, RULE_skip_distance = 10, RULE_window_width = 11, RULE_time_interval = 12, 
+		RULE_time_amount = 13, RULE_time_unit = 14;
+	public static final String[] ruleNames = {
+		"window", "window_expression", "excluding_specifier", "including_specifier", 
+		"specifier", "specifier_arg_list", "day_specifier", "identifier", "specifier_list", 
+		"duration", "skip_distance", "window_width", "time_interval", "time_amount", 
+		"time_unit"
+	};
+
+	private static final String[] _LITERAL_NAMES = {
+		null, "','", "':'"
+	};
+	private static final String[] _SYMBOLIC_NAMES = {
+		null, "COMMA", "COLON", "WINDOW", "INCLUDE", "EXCLUDE", "NOW", "FROM", 
+		"EVERY", "TO", "AGO", "NUMBER", "IDENTIFIER", "DAY_SPECIFIER", "TIME_UNIT", 
+		"WS"
+	};
+	public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES);
+
+	/**
+	 * @deprecated Use {@link #VOCABULARY} instead.
+	 */
+	@Deprecated
+	public static final String[] tokenNames;
+	static {
+		tokenNames = new String[_SYMBOLIC_NAMES.length];
+		for (int i = 0; i < tokenNames.length; i++) {
+			tokenNames[i] = VOCABULARY.getLiteralName(i);
+			if (tokenNames[i] == null) {
+				tokenNames[i] = VOCABULARY.getSymbolicName(i);
+			}
+
+			if (tokenNames[i] == null) {
+				tokenNames[i] = "<INVALID>";
+			}
+		}
+	}
+
+	@Override
+	@Deprecated
+	public String[] getTokenNames() {
+		return tokenNames;
+	}
+
+	@Override
+
+	public Vocabulary getVocabulary() {
+		return VOCABULARY;
+	}
+
+	@Override
+	public String getGrammarFileName() { return "Window.g4"; }
+
+	@Override
+	public String[] getRuleNames() { return ruleNames; }
+
+	@Override
+	public String getSerializedATN() { return _serializedATN; }
+
+	@Override
+	public ATN getATN() { return _ATN; }
+
+	public WindowParser(TokenStream input) {
+		super(input);
+		_interp = new ParserATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache);
+	}
+	public static class WindowContext extends ParserRuleContext {
+		public Window_expressionContext window_expression() {
+			return getRuleContext(Window_expressionContext.class,0);
+		}
+		public TerminalNode EOF() { return getToken(WindowParser.EOF, 0); }
+		public WindowContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_window; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterWindow(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitWindow(this);
+		}
+	}
+
+	public final WindowContext window() throws RecognitionException {
+		WindowContext _localctx = new WindowContext(_ctx, getState());
+		enterRule(_localctx, 0, RULE_window);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(30);
+			window_expression();
+			setState(31);
+			match(EOF);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Window_expressionContext extends ParserRuleContext {
+		public Window_expressionContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_window_expression; }
+	 
+		public Window_expressionContext() { }
+		public void copyFrom(Window_expressionContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class RepeatingWindowContext extends Window_expressionContext {
+		public Window_widthContext window_width() {
+			return getRuleContext(Window_widthContext.class,0);
+		}
+		public Skip_distanceContext skip_distance() {
+			return getRuleContext(Skip_distanceContext.class,0);
+		}
+		public DurationContext duration() {
+			return getRuleContext(DurationContext.class,0);
+		}
+		public Including_specifierContext including_specifier() {
+			return getRuleContext(Including_specifierContext.class,0);
+		}
+		public Excluding_specifierContext excluding_specifier() {
+			return getRuleContext(Excluding_specifierContext.class,0);
+		}
+		public RepeatingWindowContext(Window_expressionContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterRepeatingWindow(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitRepeatingWindow(this);
+		}
+	}
+	public static class DenseWindowContext extends Window_expressionContext {
+		public DurationContext duration() {
+			return getRuleContext(DurationContext.class,0);
+		}
+		public DenseWindowContext(Window_expressionContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterDenseWindow(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitDenseWindow(this);
+		}
+	}
+	public static class NonRepeatingWindowContext extends Window_expressionContext {
+		public Window_widthContext window_width() {
+			return getRuleContext(Window_widthContext.class,0);
+		}
+		public Including_specifierContext including_specifier() {
+			return getRuleContext(Including_specifierContext.class,0);
+		}
+		public Excluding_specifierContext excluding_specifier() {
+			return getRuleContext(Excluding_specifierContext.class,0);
+		}
+		public NonRepeatingWindowContext(Window_expressionContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterNonRepeatingWindow(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitNonRepeatingWindow(this);
+		}
+	}
+
+	public final Window_expressionContext window_expression() throws RecognitionException {
+		Window_expressionContext _localctx = new Window_expressionContext(_ctx, getState());
+		enterRule(_localctx, 2, RULE_window_expression);
+		int _la;
+		try {
+			setState(50);
+			switch ( getInterpreter().adaptivePredict(_input,4,_ctx) ) {
+			case 1:
+				_localctx = new NonRepeatingWindowContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(33);
+				window_width();
+				setState(35);
+				_la = _input.LA(1);
+				if (_la==INCLUDE) {
+					{
+					setState(34);
+					including_specifier();
+					}
+				}
+
+				setState(38);
+				_la = _input.LA(1);
+				if (_la==EXCLUDE) {
+					{
+					setState(37);
+					excluding_specifier();
+					}
+				}
+
+				}
+				break;
+			case 2:
+				_localctx = new RepeatingWindowContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(40);
+				window_width();
+				setState(41);
+				skip_distance();
+				setState(42);
+				duration();
+				setState(44);
+				_la = _input.LA(1);
+				if (_la==INCLUDE) {
+					{
+					setState(43);
+					including_specifier();
+					}
+				}
+
+				setState(47);
+				_la = _input.LA(1);
+				if (_la==EXCLUDE) {
+					{
+					setState(46);
+					excluding_specifier();
+					}
+				}
+
+				}
+				break;
+			case 3:
+				_localctx = new DenseWindowContext(_localctx);
+				enterOuterAlt(_localctx, 3);
+				{
+				setState(49);
+				duration();
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Excluding_specifierContext extends ParserRuleContext {
+		public TerminalNode EXCLUDE() { return getToken(WindowParser.EXCLUDE, 0); }
+		public Specifier_listContext specifier_list() {
+			return getRuleContext(Specifier_listContext.class,0);
+		}
+		public Excluding_specifierContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_excluding_specifier; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterExcluding_specifier(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitExcluding_specifier(this);
+		}
+	}
+
+	public final Excluding_specifierContext excluding_specifier() throws RecognitionException {
+		Excluding_specifierContext _localctx = new Excluding_specifierContext(_ctx, getState());
+		enterRule(_localctx, 4, RULE_excluding_specifier);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(52);
+			match(EXCLUDE);
+			setState(53);
+			specifier_list(0);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Including_specifierContext extends ParserRuleContext {
+		public TerminalNode INCLUDE() { return getToken(WindowParser.INCLUDE, 0); }
+		public Specifier_listContext specifier_list() {
+			return getRuleContext(Specifier_listContext.class,0);
+		}
+		public Including_specifierContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_including_specifier; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterIncluding_specifier(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitIncluding_specifier(this);
+		}
+	}
+
+	public final Including_specifierContext including_specifier() throws RecognitionException {
+		Including_specifierContext _localctx = new Including_specifierContext(_ctx, getState());
+		enterRule(_localctx, 6, RULE_including_specifier);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(55);
+			match(INCLUDE);
+			setState(56);
+			specifier_list(0);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class SpecifierContext extends ParserRuleContext {
+		public Day_specifierContext day_specifier() {
+			return getRuleContext(Day_specifierContext.class,0);
+		}
+		public Specifier_arg_listContext specifier_arg_list() {
+			return getRuleContext(Specifier_arg_listContext.class,0);
+		}
+		public SpecifierContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_specifier; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterSpecifier(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitSpecifier(this);
+		}
+	}
+
+	public final SpecifierContext specifier() throws RecognitionException {
+		SpecifierContext _localctx = new SpecifierContext(_ctx, getState());
+		enterRule(_localctx, 8, RULE_specifier);
+		try {
+			setState(62);
+			switch ( getInterpreter().adaptivePredict(_input,5,_ctx) ) {
+			case 1:
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(58);
+				day_specifier();
+				}
+				break;
+			case 2:
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(59);
+				day_specifier();
+				setState(60);
+				specifier_arg_list();
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Specifier_arg_listContext extends ParserRuleContext {
+		public IdentifierContext identifier() {
+			return getRuleContext(IdentifierContext.class,0);
+		}
+		public Specifier_arg_listContext specifier_arg_list() {
+			return getRuleContext(Specifier_arg_listContext.class,0);
+		}
+		public Specifier_arg_listContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_specifier_arg_list; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterSpecifier_arg_list(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitSpecifier_arg_list(this);
+		}
+	}
+
+	public final Specifier_arg_listContext specifier_arg_list() throws RecognitionException {
+		Specifier_arg_listContext _localctx = new Specifier_arg_listContext(_ctx, getState());
+		enterRule(_localctx, 10, RULE_specifier_arg_list);
+		try {
+			setState(68);
+			switch ( getInterpreter().adaptivePredict(_input,6,_ctx) ) {
+			case 1:
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(64);
+				identifier();
+				}
+				break;
+			case 2:
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(65);
+				identifier();
+				setState(66);
+				specifier_arg_list();
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Day_specifierContext extends ParserRuleContext {
+		public TerminalNode DAY_SPECIFIER() { return getToken(WindowParser.DAY_SPECIFIER, 0); }
+		public Day_specifierContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_day_specifier; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterDay_specifier(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitDay_specifier(this);
+		}
+	}
+
+	public final Day_specifierContext day_specifier() throws RecognitionException {
+		Day_specifierContext _localctx = new Day_specifierContext(_ctx, getState());
+		enterRule(_localctx, 12, RULE_day_specifier);
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(70);
+			match(DAY_SPECIFIER);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class IdentifierContext extends ParserRuleContext {
+		public TerminalNode NUMBER() { return getToken(WindowParser.NUMBER, 0); }
+		public TerminalNode IDENTIFIER() { return getToken(WindowParser.IDENTIFIER, 0); }
+		public IdentifierContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_identifier; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterIdentifier(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitIdentifier(this);
+		}
+	}
+
+	public final IdentifierContext identifier() throws RecognitionException {
+		IdentifierContext _localctx = new IdentifierContext(_ctx, getState());
+		enterRule(_localctx, 14, RULE_identifier);
+		int _la;
+		try {
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(72);
+			_la = _input.LA(1);
+			if ( !(_la==NUMBER || _la==IDENTIFIER) ) {
+			_errHandler.recoverInline(this);
+			} else {
+				consume();
+			}
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Specifier_listContext extends ParserRuleContext {
+		public SpecifierContext specifier() {
+			return getRuleContext(SpecifierContext.class,0);
+		}
+		public Specifier_listContext specifier_list() {
+			return getRuleContext(Specifier_listContext.class,0);
+		}
+		public TerminalNode COMMA() { return getToken(WindowParser.COMMA, 0); }
+		public Specifier_listContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_specifier_list; }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterSpecifier_list(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitSpecifier_list(this);
+		}
+	}
+
+	public final Specifier_listContext specifier_list() throws RecognitionException {
+		return specifier_list(0);
+	}
+
+	private Specifier_listContext specifier_list(int _p) throws RecognitionException {
+		ParserRuleContext _parentctx = _ctx;
+		int _parentState = getState();
+		Specifier_listContext _localctx = new Specifier_listContext(_ctx, _parentState);
+		Specifier_listContext _prevctx = _localctx;
+		int _startState = 16;
+		enterRecursionRule(_localctx, 16, RULE_specifier_list, _p);
+		try {
+			int _alt;
+			enterOuterAlt(_localctx, 1);
+			{
+			{
+			setState(75);
+			specifier();
+			}
+			_ctx.stop = _input.LT(-1);
+			setState(82);
+			_errHandler.sync(this);
+			_alt = getInterpreter().adaptivePredict(_input,7,_ctx);
+			while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
+				if ( _alt==1 ) {
+					if ( _parseListeners!=null ) triggerExitRuleEvent();
+					_prevctx = _localctx;
+					{
+					{
+					_localctx = new Specifier_listContext(_parentctx, _parentState);
+					pushNewRecursionContext(_localctx, _startState, RULE_specifier_list);
+					setState(77);
+					if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
+					setState(78);
+					match(COMMA);
+					setState(79);
+					specifier();
+					}
+					} 
+				}
+				setState(84);
+				_errHandler.sync(this);
+				_alt = getInterpreter().adaptivePredict(_input,7,_ctx);
+			}
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			unrollRecursionContexts(_parentctx);
+		}
+		return _localctx;
+	}
+
+	public static class DurationContext extends ParserRuleContext {
+		public DurationContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_duration; }
+	 
+		public DurationContext() { }
+		public void copyFrom(DurationContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class FromToDurationContext extends DurationContext {
+		public TerminalNode FROM() { return getToken(WindowParser.FROM, 0); }
+		public List<Time_intervalContext> time_interval() {
+			return getRuleContexts(Time_intervalContext.class);
+		}
+		public Time_intervalContext time_interval(int i) {
+			return getRuleContext(Time_intervalContext.class,i);
+		}
+		public TerminalNode TO() { return getToken(WindowParser.TO, 0); }
+		public List<TerminalNode> AGO() { return getTokens(WindowParser.AGO); }
+		public TerminalNode AGO(int i) {
+			return getToken(WindowParser.AGO, i);
+		}
+		public FromToDurationContext(DurationContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterFromToDuration(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitFromToDuration(this);
+		}
+	}
+	public static class FromDurationContext extends DurationContext {
+		public TerminalNode FROM() { return getToken(WindowParser.FROM, 0); }
+		public Time_intervalContext time_interval() {
+			return getRuleContext(Time_intervalContext.class,0);
+		}
+		public TerminalNode AGO() { return getToken(WindowParser.AGO, 0); }
+		public FromDurationContext(DurationContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterFromDuration(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitFromDuration(this);
+		}
+	}
+
+	public final DurationContext duration() throws RecognitionException {
+		DurationContext _localctx = new DurationContext(_ctx, getState());
+		enterRule(_localctx, 18, RULE_duration);
+		int _la;
+		try {
+			setState(100);
+			switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) {
+			case 1:
+				_localctx = new FromToDurationContext(_localctx);
+				enterOuterAlt(_localctx, 1);
+				{
+				setState(85);
+				match(FROM);
+				setState(86);
+				time_interval();
+				setState(88);
+				_la = _input.LA(1);
+				if (_la==AGO) {
+					{
+					setState(87);
+					match(AGO);
+					}
+				}
+
+				setState(90);
+				match(TO);
+				setState(91);
+				time_interval();
+				setState(93);
+				_la = _input.LA(1);
+				if (_la==AGO) {
+					{
+					setState(92);
+					match(AGO);
+					}
+				}
+
+				}
+				break;
+			case 2:
+				_localctx = new FromDurationContext(_localctx);
+				enterOuterAlt(_localctx, 2);
+				{
+				setState(95);
+				match(FROM);
+				setState(96);
+				time_interval();
+				setState(98);
+				_la = _input.LA(1);
+				if (_la==AGO) {
+					{
+					setState(97);
+					match(AGO);
+					}
+				}
+
+				}
+				break;
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Skip_distanceContext extends ParserRuleContext {
+		public Skip_distanceContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_skip_distance; }
+	 
+		public Skip_distanceContext() { }
+		public void copyFrom(Skip_distanceContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class SkipDistanceContext extends Skip_distanceContext {
+		public TerminalNode EVERY() { return getToken(WindowParser.EVERY, 0); }
+		public Time_intervalContext time_interval() {
+			return getRuleContext(Time_intervalContext.class,0);
+		}
+		public SkipDistanceContext(Skip_distanceContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterSkipDistance(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitSkipDistance(this);
+		}
+	}
+
+	public final Skip_distanceContext skip_distance() throws RecognitionException {
+		Skip_distanceContext _localctx = new Skip_distanceContext(_ctx, getState());
+		enterRule(_localctx, 20, RULE_skip_distance);
+		try {
+			_localctx = new SkipDistanceContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(102);
+			match(EVERY);
+			setState(103);
+			time_interval();
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Window_widthContext extends ParserRuleContext {
+		public Window_widthContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_window_width; }
+	 
+		public Window_widthContext() { }
+		public void copyFrom(Window_widthContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class WindowWidthContext extends Window_widthContext {
+		public Time_intervalContext time_interval() {
+			return getRuleContext(Time_intervalContext.class,0);
+		}
+		public TerminalNode WINDOW() { return getToken(WindowParser.WINDOW, 0); }
+		public WindowWidthContext(Window_widthContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterWindowWidth(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitWindowWidth(this);
+		}
+	}
+
+	public final Window_widthContext window_width() throws RecognitionException {
+		Window_widthContext _localctx = new Window_widthContext(_ctx, getState());
+		enterRule(_localctx, 22, RULE_window_width);
+		int _la;
+		try {
+			_localctx = new WindowWidthContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(105);
+			time_interval();
+			setState(107);
+			_la = _input.LA(1);
+			if (_la==WINDOW) {
+				{
+				setState(106);
+				match(WINDOW);
+				}
+			}
+
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Time_intervalContext extends ParserRuleContext {
+		public Time_intervalContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_time_interval; }
+	 
+		public Time_intervalContext() { }
+		public void copyFrom(Time_intervalContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class TimeIntervalContext extends Time_intervalContext {
+		public Time_amountContext time_amount() {
+			return getRuleContext(Time_amountContext.class,0);
+		}
+		public Time_unitContext time_unit() {
+			return getRuleContext(Time_unitContext.class,0);
+		}
+		public TimeIntervalContext(Time_intervalContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterTimeInterval(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitTimeInterval(this);
+		}
+	}
+
+	public final Time_intervalContext time_interval() throws RecognitionException {
+		Time_intervalContext _localctx = new Time_intervalContext(_ctx, getState());
+		enterRule(_localctx, 24, RULE_time_interval);
+		try {
+			_localctx = new TimeIntervalContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(109);
+			time_amount();
+			setState(110);
+			time_unit();
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Time_amountContext extends ParserRuleContext {
+		public Time_amountContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_time_amount; }
+	 
+		public Time_amountContext() { }
+		public void copyFrom(Time_amountContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class TimeAmountContext extends Time_amountContext {
+		public TerminalNode NUMBER() { return getToken(WindowParser.NUMBER, 0); }
+		public TimeAmountContext(Time_amountContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterTimeAmount(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitTimeAmount(this);
+		}
+	}
+
+	public final Time_amountContext time_amount() throws RecognitionException {
+		Time_amountContext _localctx = new Time_amountContext(_ctx, getState());
+		enterRule(_localctx, 26, RULE_time_amount);
+		try {
+			_localctx = new TimeAmountContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(112);
+			match(NUMBER);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public static class Time_unitContext extends ParserRuleContext {
+		public Time_unitContext(ParserRuleContext parent, int invokingState) {
+			super(parent, invokingState);
+		}
+		@Override public int getRuleIndex() { return RULE_time_unit; }
+	 
+		public Time_unitContext() { }
+		public void copyFrom(Time_unitContext ctx) {
+			super.copyFrom(ctx);
+		}
+	}
+	public static class TimeUnitContext extends Time_unitContext {
+		public TerminalNode TIME_UNIT() { return getToken(WindowParser.TIME_UNIT, 0); }
+		public TimeUnitContext(Time_unitContext ctx) { copyFrom(ctx); }
+		@Override
+		public void enterRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).enterTimeUnit(this);
+		}
+		@Override
+		public void exitRule(ParseTreeListener listener) {
+			if ( listener instanceof WindowListener ) ((WindowListener)listener).exitTimeUnit(this);
+		}
+	}
+
+	public final Time_unitContext time_unit() throws RecognitionException {
+		Time_unitContext _localctx = new Time_unitContext(_ctx, getState());
+		enterRule(_localctx, 28, RULE_time_unit);
+		try {
+			_localctx = new TimeUnitContext(_localctx);
+			enterOuterAlt(_localctx, 1);
+			{
+			setState(114);
+			match(TIME_UNIT);
+			}
+		}
+		catch (RecognitionException re) {
+			_localctx.exception = re;
+			_errHandler.reportError(this, re);
+			_errHandler.recover(this, re);
+		}
+		finally {
+			exitRule();
+		}
+		return _localctx;
+	}
+
+	public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) {
+		switch (ruleIndex) {
+		case 8:
+			return specifier_list_sempred((Specifier_listContext)_localctx, predIndex);
+		}
+		return true;
+	}
+	private boolean specifier_list_sempred(Specifier_listContext _localctx, int predIndex) {
+		switch (predIndex) {
+		case 0:
+			return precpred(_ctx, 1);
+		}
+		return true;
+	}
+
+	public static final String _serializedATN =
+		"\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3\21w\4\2\t\2\4\3\t"+
+		"\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13\t\13\4"+
+		"\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\3\2\3\2\3\2\3\3\3\3\5\3&"+
+		"\n\3\3\3\5\3)\n\3\3\3\3\3\3\3\3\3\5\3/\n\3\3\3\5\3\62\n\3\3\3\5\3\65\n"+
+		"\3\3\4\3\4\3\4\3\5\3\5\3\5\3\6\3\6\3\6\3\6\5\6A\n\6\3\7\3\7\3\7\3\7\5"+
+		"\7G\n\7\3\b\3\b\3\t\3\t\3\n\3\n\3\n\3\n\3\n\3\n\7\nS\n\n\f\n\16\nV\13"+
+		"\n\3\13\3\13\3\13\5\13[\n\13\3\13\3\13\3\13\5\13`\n\13\3\13\3\13\3\13"+
+		"\5\13e\n\13\5\13g\n\13\3\f\3\f\3\f\3\r\3\r\5\rn\n\r\3\16\3\16\3\16\3\17"+
+		"\3\17\3\20\3\20\3\20\2\3\22\21\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36"+
+		"\2\3\3\2\r\16u\2 \3\2\2\2\4\64\3\2\2\2\6\66\3\2\2\2\b9\3\2\2\2\n@\3\2"+
+		"\2\2\fF\3\2\2\2\16H\3\2\2\2\20J\3\2\2\2\22L\3\2\2\2\24f\3\2\2\2\26h\3"+
+		"\2\2\2\30k\3\2\2\2\32o\3\2\2\2\34r\3\2\2\2\36t\3\2\2\2 !\5\4\3\2!\"\7"+
+		"\2\2\3\"\3\3\2\2\2#%\5\30\r\2$&\5\b\5\2%$\3\2\2\2%&\3\2\2\2&(\3\2\2\2"+
+		"\')\5\6\4\2(\'\3\2\2\2()\3\2\2\2)\65\3\2\2\2*+\5\30\r\2+,\5\26\f\2,.\5"+
+		"\24\13\2-/\5\b\5\2.-\3\2\2\2./\3\2\2\2/\61\3\2\2\2\60\62\5\6\4\2\61\60"+
+		"\3\2\2\2\61\62\3\2\2\2\62\65\3\2\2\2\63\65\5\24\13\2\64#\3\2\2\2\64*\3"+
+		"\2\2\2\64\63\3\2\2\2\65\5\3\2\2\2\66\67\7\7\2\2\678\5\22\n\28\7\3\2\2"+
+		"\29:\7\6\2\2:;\5\22\n\2;\t\3\2\2\2<A\5\16\b\2=>\5\16\b\2>?\5\f\7\2?A\3"+
+		"\2\2\2@<\3\2\2\2@=\3\2\2\2A\13\3\2\2\2BG\5\20\t\2CD\5\20\t\2DE\5\f\7\2"+
+		"EG\3\2\2\2FB\3\2\2\2FC\3\2\2\2G\r\3\2\2\2HI\7\17\2\2I\17\3\2\2\2JK\t\2"+
+		"\2\2K\21\3\2\2\2LM\b\n\1\2MN\5\n\6\2NT\3\2\2\2OP\f\3\2\2PQ\7\3\2\2QS\5"+
+		"\n\6\2RO\3\2\2\2SV\3\2\2\2TR\3\2\2\2TU\3\2\2\2U\23\3\2\2\2VT\3\2\2\2W"+
+		"X\7\t\2\2XZ\5\32\16\2Y[\7\f\2\2ZY\3\2\2\2Z[\3\2\2\2[\\\3\2\2\2\\]\7\13"+
+		"\2\2]_\5\32\16\2^`\7\f\2\2_^\3\2\2\2_`\3\2\2\2`g\3\2\2\2ab\7\t\2\2bd\5"+
+		"\32\16\2ce\7\f\2\2dc\3\2\2\2de\3\2\2\2eg\3\2\2\2fW\3\2\2\2fa\3\2\2\2g"+
+		"\25\3\2\2\2hi\7\n\2\2ij\5\32\16\2j\27\3\2\2\2km\5\32\16\2ln\7\5\2\2ml"+
+		"\3\2\2\2mn\3\2\2\2n\31\3\2\2\2op\5\34\17\2pq\5\36\20\2q\33\3\2\2\2rs\7"+
+		"\r\2\2s\35\3\2\2\2tu\7\20\2\2u\37\3\2\2\2\17%(.\61\64@FTZ_dfm";
+	public static final ATN _ATN =
+		new ATNDeserializer().deserialize(_serializedATN.toCharArray());
+	static {
+		_decisionToDFA = new DFA[_ATN.getNumberOfDecisions()];
+		for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) {
+			_decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DateSpecifierPredicate.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DateSpecifierPredicate.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DateSpecifierPredicate.java
new file mode 100644
index 0000000..5086719
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DateSpecifierPredicate.java
@@ -0,0 +1,82 @@
+/*
+ *
+ *  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.metron.profiler.client.window.predicates;
+
+import org.apache.commons.lang3.time.DateUtils;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+/**
+ * An inclusion/exclusion specifier which matches against a specific date.
+ * The default format is yyyy/MM/dd
+ */
+public class DateSpecifierPredicate implements Predicate<Long> {
+  final static ThreadLocal<SimpleDateFormat> FORMAT = new ThreadLocal<SimpleDateFormat>() {
+
+    @Override
+    protected SimpleDateFormat initialValue() {
+      return new SimpleDateFormat("yyyy/MM/dd");
+    }
+  };
+  Date date;
+
+  /**
+   * Create a predicate given a date and (optionally) a format.  If no format is specified, it is assumed to be
+   * yyyy/MM/dd
+   * @param args
+   */
+  public DateSpecifierPredicate(List<String> args) {
+    if(args.size() == 1) {
+      //just the date, use the default.
+      try {
+        date = FORMAT.get().parse(args.get(0));
+      } catch (ParseException e) {
+        throw new IllegalStateException("Unable to process " + args.get(0) + " as a date using " + FORMAT.get().toPattern());
+      }
+    }
+    else if(args.size() == 0){
+      throw new IllegalStateException("You must specify at least a date and optionally a format");
+    }
+    else {
+      String dateStr = args.get(0);
+      String format =  args.get(1);
+      try {
+        date = new SimpleDateFormat(format).parse(dateStr);
+      } catch (ParseException e) {
+        throw new IllegalStateException("Unable to process " + dateStr + " as a date using " + format);
+      }
+    }
+  }
+
+  /**
+   * Returns true if the timestamp happens on the specified day and false otherwise.
+   * @param ts
+   * @return
+   */
+  @Override
+  public boolean test(Long ts) {
+    return DateUtils.isSameDay(new Date(ts), date);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DayPredicates.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DayPredicates.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DayPredicates.java
new file mode 100644
index 0000000..8025371
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/DayPredicates.java
@@ -0,0 +1,113 @@
+/*
+ *
+ *  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.metron.profiler.client.window.predicates;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * The inclusion/exclusion selector predicates.  The enum name is intended to match the inclusion/exclusion selector
+ * name in the window selector grammar.
+ */
+public enum DayPredicates {
+  /**
+   * True if the day is sunday, false otherwise.
+   */
+  SUNDAY( x -> dayOfWeekPredicate(1)),
+  /**
+   * True if the day is monday, false otherwise.
+   */
+  MONDAY( x -> dayOfWeekPredicate(2)),
+  /**
+   * True if the day is tuesday, false otherwise.
+   */
+  TUESDAY( x -> dayOfWeekPredicate(3)),
+  /**
+   * True if the day is wednesday, false otherwise.
+   */
+  WEDNESDAY( x -> dayOfWeekPredicate(4)),
+  /**
+   * True if the day is thursday, false otherwise.
+   */
+  THURSDAY( x -> dayOfWeekPredicate(5)),
+  /**
+   * True if the day is friday, false otherwise.
+   */
+  FRIDAY( x -> dayOfWeekPredicate(6)),
+  /**
+   * True if the day is saturday, false otherwise.
+   */
+  SATURDAY( x -> dayOfWeekPredicate(7)),
+  /**
+   * True if the day is a weekday, false otherwise.
+   */
+  WEEKDAY( x -> (ts -> {
+    int dow = toCalendar(ts).get(Calendar.DAY_OF_WEEK);
+    return dow > 1 && dow < 7;
+  })),
+  /**
+   * True if the day is a weekend, false otherwise.
+   */
+  WEEKEND( x -> (ts -> {
+    int dow = toCalendar(ts).get(Calendar.DAY_OF_WEEK);
+    return dow == 1 || dow == 7;
+  })),
+  /**
+   * True if the day is a holiday, false otherwise.
+   */
+  HOLIDAY(x -> new HolidaysPredicate(x)),
+  /**
+   * True if the day is a specifie ddate, false otherwise.
+   */
+  DATE( x -> new DateSpecifierPredicate(x))
+  ;
+  Function<List<String>, Predicate<Long>> predicateCreator;
+  DayPredicates(Function<List<String>, Predicate<Long>> predicate) {
+    this.predicateCreator = predicate;
+  }
+
+  private static Calendar toCalendar(Long ts) {
+    Calendar c = Calendar.getInstance();
+    c.setTime(new Date(ts));
+    return c;
+  }
+
+  public static int getDayOfWeek(Long ts) {
+    return toCalendar(ts).get(Calendar.DAY_OF_WEEK);
+  }
+
+  public static Predicate<Long> dayOfWeekPredicate(int dayOfWeek) {
+    return ts -> getDayOfWeek(ts) == dayOfWeek;
+  }
+
+  /**
+   * Create a Predicate given a set of arguments.
+   * @param name
+   * @param arg
+   * @return
+   */
+  public static Predicate<Long> create(String name, List<String> arg) {
+    return DayPredicates.valueOf(name).predicateCreator.apply(arg);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/HolidaysPredicate.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/HolidaysPredicate.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/HolidaysPredicate.java
new file mode 100644
index 0000000..4705d42
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/predicates/HolidaysPredicate.java
@@ -0,0 +1,87 @@
+/*
+ *
+ *  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.metron.profiler.client.window.predicates;
+
+import de.jollyday.HolidayCalendar;
+import de.jollyday.HolidayManager;
+import de.jollyday.ManagerParameters;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Predicate;
+
+/**
+ * A inclusion/exclusion selector predicate that returns true if a timestamp falls on a holiday and
+ * false otherwise.
+ */
+public class HolidaysPredicate implements Predicate<Long> {
+  HolidayManager manager;
+  String[] args;
+
+  /**
+   * Create a holidays predicate.  The arguments are the hierarchical specifier for the holidays
+   * (see https://github.com/svendiedrichsen/jollyday/tree/master/src/main/resources/holidays for the hierarchies for your
+   * supported locales).  The first param is the locale.
+   * @param args
+   */
+  public HolidaysPredicate(List<String> args) {
+    if(args == null || args.size() == 0) {
+      this.manager = HolidayManager.getInstance();
+      this.args = new String[]{};
+    }
+    else {
+      String code = args.get(0);
+      this.args = args.size() == 1 ? new String[]{} : new String[args.size() - 1];
+      Optional<HolidayCalendar> calendar = getCalendar(code);
+      if(calendar.isPresent()) {
+        this.manager = HolidayManager.getInstance(ManagerParameters.create(calendar.get()));
+      }
+      else {
+        this.manager = HolidayManager.getInstance(ManagerParameters.create(code));
+      }
+      for (int i = 1; i < args.size(); ++i) {
+        this.args[i - 1] = args.get(i);
+      }
+    }
+  }
+
+  private static Optional<HolidayCalendar> getCalendar(String code) {
+    for(HolidayCalendar cal : HolidayCalendar.values()) {
+      if(cal.getId().equalsIgnoreCase(code) || cal.name().equalsIgnoreCase(code)) {
+        return Optional.of(cal);
+      }
+    }
+    return Optional.empty();
+  }
+
+  /**
+   * True if the timestamp falls on a holiday as specified or false otherwise.
+   * @param ts
+   * @return
+   */
+  @Override
+  public boolean test(Long ts) {
+    Calendar c = Calendar.getInstance();
+    c.setTime(new Date(ts));
+    return manager.isHoliday(c, args);
+  }
+}



[3/3] incubator-metron git commit: METRON-690: Create a DSL-based timestamp lookup for profiler to enable sparse windows closes apache/incubator-metron#450

Posted by ce...@apache.org.
METRON-690: Create a DSL-based timestamp lookup for profiler to enable sparse windows closes apache/incubator-metron#450


Project: http://git-wip-us.apache.org/repos/asf/incubator-metron/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-metron/commit/84d34719
Tree: http://git-wip-us.apache.org/repos/asf/incubator-metron/tree/84d34719
Diff: http://git-wip-us.apache.org/repos/asf/incubator-metron/diff/84d34719

Branch: refs/heads/master
Commit: 84d347195a817e70fb792b4b240151b7654b1a9c
Parents: 7abd7e8
Author: cstella <ce...@gmail.com>
Authored: Fri Feb 24 15:53:05 2017 -0500
Committer: cstella <ce...@gmail.com>
Committed: Fri Feb 24 15:53:05 2017 -0500

----------------------------------------------------------------------
 dependencies_with_url.csv                       |    3 +-
 .../metron-profiler-client/README.md            |  287 ++++-
 metron-analytics/metron-profiler-client/pom.xml |   27 +
 .../profiler/client/window/generated/Window.g4  |  136 +++
 .../src/main/java/Window.tokens                 |   17 +
 .../src/main/java/WindowLexer.tokens            |   17 +
 .../client/stellar/IntervalPredicate.java       |  106 ++
 .../profiler/client/stellar/WindowLookback.java |  107 ++
 .../metron/profiler/client/window/Window.java   |  159 +++
 .../profiler/client/window/WindowProcessor.java |  422 +++++++
 .../window/generated/WindowBaseListener.java    |  276 +++++
 .../client/window/generated/WindowLexer.java    |  477 ++++++++
 .../client/window/generated/WindowListener.java |  231 ++++
 .../client/window/generated/WindowParser.java   | 1096 ++++++++++++++++++
 .../predicates/DateSpecifierPredicate.java      |   82 ++
 .../client/window/predicates/DayPredicates.java |  113 ++
 .../window/predicates/HolidaysPredicate.java    |   87 ++
 .../client/stellar/IntervalPredicateTest.java   |   89 ++
 .../client/stellar/WindowLookbackTest.java      |  207 ++++
 .../client/window/WindowProcessorTest.java      |  314 +++++
 .../apache/metron/profiler/ProfilePeriod.java   |   12 +-
 metron-platform/metron-common/README.md         |    9 +
 metron-platform/metron-common/pom.xml           |    5 +-
 .../apache/metron/common/dsl/GrammarUtils.java  |  133 +++
 .../org/apache/metron/common/dsl/Token.java     |    2 +-
 pom.xml                                         |    1 +
 26 files changed, 4361 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/dependencies_with_url.csv
----------------------------------------------------------------------
diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv
index 3640b14..fded057 100644
--- a/dependencies_with_url.csv
+++ b/dependencies_with_url.csv
@@ -221,4 +221,5 @@ org.yaml:snakeyaml:jar:1.15:compile,Apache License Version 2.0,http://www.snakey
 ring-cors:ring-cors:jar:0.1.5:compile,Eclipse Public License 1.0,https://github.com/r0man/ring-cors
 xerces:xercesImpl:jar:2.9.1:compile,ASLv2,http://xerces.apache.org/xerces2-j
 xml-apis:xml-apis:jar:1.3.04:compile,ASLv2,http://xml.apache.org/commons/components/external/
-
+de.jollyday:jollyday:jar:0.5.2:compile,ASLv2,http://jollyday.sourceforge.net/license.html
+org.threeten:threeten-extra:jar:1.0:compile,BSD,http://www.threeten.org/threeten-extra/license.html

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/README.md b/metron-analytics/metron-profiler-client/README.md
index 94da919..656bb25 100644
--- a/metron-analytics/metron-profiler-client/README.md
+++ b/metron-analytics/metron-profiler-client/README.md
@@ -4,27 +4,9 @@ This project provides a client API for accessing the profiles generated by the [
 
 ## Stellar Client API
 
-The following are usage examples that show how the Stellar API can be used to read profiles generated by the [Metron Profiler](../metron-profiler).  This API would be used in conjunction with other Stellar functions like [`MAAS_MODEL_APPLY`](../../metron-platform/metron-common#maas_model_apply) to perform model scoring on streaming data.
-
-These examples assume a profile has been defined called 'snort-alerts' that tracks the number of Snort alerts associated with an IP address over time.  The profile definition might look similar to the following.
-
-```
-{
-  "profiles": [
-    {
-      "profile": "snort-alerts",
-      "foreach": "ip_src_addr",
-      "onlyif":  "source.type == 'snort'",
-      "update":  { "s": "STATS_ADD(s, 1)" },
-      "result":  "STATS_MEAN(s)"
-    }
-  ]
-}
-```
+### `PROFILE_GET`
 
-During model scoring the entity being scored, in this case a particular IP address, will be known.  The following examples shows how this profile data might be retrieved.
-
-The Stellar client consists of the `PROFILE_GET` command, which takes the following arguments:
+The `PROFILE_GET` command allows you to select all of the profile measurements written.  This command takes the following arguments:
 ```
 REQUIRED:
     profile - The name of the profile
@@ -41,20 +23,7 @@ There is an older calling format where `groups_list` is specified as a sequence
 
 The `periods` field is (likely) the output of another Stellar function which defines the times to include.
 
-`PROFILE_FIXED`: The profiler periods associated with a fixed lookback starting from now.  These are ProfilePeriod objects.
-```
-REQUIRED:
-    durationAgo - How long ago should values be retrieved from?
-    units - The units of 'durationAgo'.
-OPTIONAL:
-    config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter
-            of the same name. Default is the empty Map, meaning no overrides.
-
-e.g. To retrieve all the profiles for the last 5 hours.  PROFILE_GET('profile', 'entity', PROFILE_FIXED(5, 'HOURS'))
-```
-
-
-### Groups_list argument
+#### Groups_list argument
 The `groups_list` argument in the client must exactly correspond to the [`groupBy`](../metron-profiler#groupby) configuration in the profile definition.  If `groupBy` was not used in the profile, `groups_list` must be empty in the client.  If `groupBy` was used in the profile, then the client `groups_list` is <b>not</b> optional; it must be the same length as the `groupBy` list, and specify exactly one selected group value for each `groupBy` criterion, in the same order.  For example:
 ```
 If in Profile, the groupBy criteria are:  [ \u201cDAY_OF_WEEK()\u201d, \u201cURL_TO_PORT()\u201d ]
@@ -62,7 +31,7 @@ Then in PROFILE_GET, an allowed groups value would be:  [ \u201c3\u201d, \u201c8080\u201d ]
 which will select only records from Tuesdays with port number 8080.
 ```
 
-### Configuration and the config_overrides argument
+#### Configuration and the config_overrides argument
 
 By default, the Profiler creates profiles with a period duration of 15 minutes. This means that data is accumulated, summarized and flushed every 15 minutes. 
 The Client API must also have knowledge of this duration to correctly retrieve the profile data. If the Client is expecting 15 minute periods, it will not be 
@@ -91,37 +60,269 @@ want to change the global Client configuration so as not to disrupt the work of
 | profiler.client.salt.divisor          | The salt divisor used to store profile data.                                                                                       | Optional | 1000     |
 | hbase.provider.impl                   | The name of the HBaseTableProvider implementation class.                                                                           | Optional |          |
 
+
+### Profile Selectors
+
+You will notice that the third argument for `PROFILE_GET` is a list of `ProfilePeriod` objects.  This list is expected to
+be produced by another Stellar function.  There are a couple options available.
+
+#### `PROFILE_FIXED`
+
+The profiler periods associated with a fixed lookback starting from now.  These are ProfilePeriod objects.
+```
+REQUIRED:
+    durationAgo - How long ago should values be retrieved from?
+    units - The units of 'durationAgo'.
+OPTIONAL:
+    config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter
+            of the same name. Default is the empty Map, meaning no overrides.
+
+e.g. To retrieve all the profiles for the last 5 hours.  PROFILE_GET('profile', 'entity', PROFILE_FIXED(5, 'HOURS'))
+```
+
+Note that the `config_overrides` parameter operates exactly as the `config_overrides` argument in `PROFILE_GET`.
+The only available parameters for override are:
+* `profiler.client.period.duration` 
+* `profiler.client.period.duration.units`
+
+#### `PROFILE_WINDOW`
+
+`PROFILE_WINDOW` is intended to provide a finer-level of control over selecting windows for profiles:
+* Specify windows relative to the data timestamp (see the optional `now` parameter below)
+* Specify non-contiguous windows to better handle seasonal data (e.g. the last hour for every day for the last month)
+* Specify profile output excluding holidays
+* Specify only profile output on a specific day of the week
+
+It does this by a domain specific language mimicking natural language that defines the windows excluded.
+
+```
+REQUIRED:
+    windowSelector - The statement specifying the window to select.
+    now - Optional - The timestamp to use for now.
+OPTIONAL:
+    config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter
+            of the same name. Default is the empty Map, meaning no overrides.
+
+e.g. To retrieve all the measurements written for 'profile' and 'entity' for the last hour 
+on the same weekday excluding weekends and US holidays across the last 14 days: 
+PROFILE_GET('profile', 'entity', PROFILE_WINDOW('1 hour window every 24 hours starting from 14 days ago including the current day of the week excluding weekends, holidays:us'))
+```
+
+Note that the `config_overrides` parameter operates exactly as the `config_overrides` argument in `PROFILE_GET`.
+The only available parameters for override are:
+* `profiler.client.period.duration`
+* `profiler.client.period.duration.units`
+
+##### The Profile Selector Language
+
+The domain specific language can be broken into a series of clauses, some optional
+* <a href="#Temporal_Window_Width"><span style="color:blue">Total Temporal Duration</span></a> - The total range of time in which windows may be specified
+* <a href="#InclusionExclusion_specifiers"><span style="color:red">Temporal Window Width</span></a> - How large each temporal window
+* <a href="#Skip_distance"><span style="color:green">Skip distance</span></a> (optional)- How far to skip between when one window starts and when the next begins
+* <a href="#InclusionExclusion_specifiers"><span style="color:purple">Inclusion/Exclusion specifiers</span></a> (optional) - The set of specifiers to further filter the window
+
+One *must* specify either a total temporal duration or a temporal window width.
+The remaining clauses are optional.
+During the course of the following discussion, we will color code the clauses in the examples and link them
+to the relevant section for more detail.
+
+From a high level, the language fits the following three forms, which are composed of the clauses above:
+
+* <a href="#Temporal_Window_Width"><span style="color:red">time_interval WINDOW?</span></a>  <a href="#InclusionExclusion_specifiers"><span style="color:purple">(INCLUDING specifier_list)? (EXCLUDING specifier_list)?</span></a>
+* <a href="#Temporal_Window_Width"><span style="color:red">time_interval WINDOW?</span></a> <a href="#Skip_distance"><span style="color:green">EVERY time_interval</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">FROM time_interval (TO time_interval)?</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">(INCLUDING specifier_list)? (EXCLUDING specifier_list)?</span></a>
+* <a href="#Total_Temporal_Duration"><span style="color:blue">FROM time_interval (TO time_interval)?</span></a>
+
+
+###### <span style="color:blue">Total Temporal Duration</span>
+
+Total temporal duration is specified by a phrase: `FROM time_interval AGO TO time_interval AGO`
+This indicates the beginning and ending of a time interval.  This is an inclusive duration.
+* `FROM` - Can be the words "from" or "starting from"
+* `time_interval` - A time amount followed by a unit (e.g. 1 hour). Fractional amounts are not supported. The unit may be "minute", "day", "hour" with any pluralization.
+* `TO` - Can be the words "until" or "to"
+* `AGO` - Optionally the word "ago"
+
+The `TO time_interval AGO` portion is optional.  If unspecified then it is expected that the time interval ends now.
+
+Due to the vagaries of the english language, the from and the to portions, if both specified, are interchangeable
+with regard to which one specifies the start and which specifies the end.  
+
+In other words "<a href="#Total_Temporal_Duration"><span style="color:blue">starting from 1 hour ago to 30 minutes ago</span></a>" and
+"<a href="#Total_Temporal_Duration"><span style="color:blue">starting from 30 minutes ago to 1 hour ago</span></a>" specify the same
+temporal duration.
+
+**Examples**
+
+* A duration starting 1 hour ago and ending now
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">from 1 hour ago</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">from 1 hour</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 1 hour ago</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 1 hour</span></a>
+* A duration starting 1 hour ago and ending 30 minutes ago: 
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">from 1 hour ago until 30 minutes ago</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">from 30 minutes ago until 1 hour ago</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 1 hour ago to 30 minutes ago</span></a>
+   * <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 1 hour to 30 minutes</span></a>
+
+###### <span style="color:red">Temporal Window Width</span>
+
+Temporal window width is the specification of a window. 
+A window is may either repeat within total temporal duration or may fill the total temporal duration.  This is an inclusive window.
+A window is specified by the phrase: `time_interval WINDOW`
+* `time_interval` - A time amount followed by a unit (e.g. 1 hour).  Fractional amounts are not supported. The unit may be "minute", "day", "hour" with any pluralization.
+* `WINDOW` - Optionally the word "window"
+
+**Examples**
+
+* A fixed window starting 2 hours ago and going until now
+  * <a href="#Temporal_Window_Width"><span style="color:red">2 hour</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">2 hours</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">2 hours window</span></a>
+* A repeating 30 minute window starting 2 hours ago and repeating every hour until now.
+This would result in 2 30-minute wide windows: 2 hours ago and 1 hour ago
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 2 hours ago</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute windows</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 2 hours ago</span></a>
+* A repeating 30 minute window starting 2 hours ago and repeating every hour until 30 minutes ago.
+This would result in 2 30-minute wide windows: 2 hours ago and 1 hour ago
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 2 hours ago until 30 minutes ago</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minutes window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 2 hours ago to 30 minutes ago</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minutes window</span></a> <a href="#Skip_distance"><span style="color:green">for every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 30 minutes ago to 2 hours ago</span></a>
+
+###### <span style="color:green">Skip distance</span>
+
+Skip distance is the amount of time between temporal window beginnings that the next window starts.
+It is, in effect, the window period.  
+
+It is specified by the phrase `EVERY time_interval`
+* `time_interval` - A time amount followed by a unit (e.g. 1 hour).  Fractional amounts are not supported. The unit may be "minute", "day", "hour" with any pluralization. 
+* `EVERY` - The word/phrase "every" or "for every"
+
+**Examples**
+
+* A repeating 30 minute window starting 2 hours ago and repeating every hour until now.
+This would result in 2 30-minute wide windows: 2 hours ago and 1 hour ago
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 2 hours ago </span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minutes window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 2 hours ago </span></a>
+* A repeating 30 minute window starting 2 hours ago and repeating every hour until 30 minutes ago.
+This would result in 2 30-minute wide windows: 2 hours ago and 1 hour ago
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">starting from 2 hours ago until 30 minutes ago</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minutes window</span></a> <a href="#Skip_distance"><span style="color:green">every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 2 hours ago to 30 minutes ago</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minutes window</span></a> <a href="#Skip_distance"><span style="color:green">for every 1 hour</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 30 minutes ago to 2 hours ago</span></a>
+
+###### <span style="color:purple">Inclusion/Exclusion specifiers</span>
+Inclusion and Exclusion specifiers operate as filters on the set of windows.
+They operate on the window beginning timestamp.
+
+For inclusion specifiers, windows who are passed by _any_ of the set of inclusion specifiers are included.  
+inclusion specifiers. Similarly, windows who are passed by _any_ of the set of exclusion specifiers are excluded.
+Exclusion specifiers trump inclusion specifiers.
+
+Specifiers follow one of the following formats depending on if it is an inclusion or exclusion specifier:
+* `INCLUSION specifier, specifier, ...`
+  * `INCLUSION` can be "include", "includes" or "including"
+* `EXCLUSION specifier, specifier, ...`
+  * `EXCLUSION` can be "exclude", "excludes" or "excluding"
+
+
+The specifiers are a set of fixed specifiers available as part of the language:
+* Fixed day of week-based specifiers - includes or excludes if the window is on the specified day of the week
+  * "monday" or "mondays"
+  * "tuesday" or "tuesdays"
+  * "wednesday" or "wednesdays"
+  * "thursday" or "thursdays"
+  * "friday" or "fridays"
+  * "saturday" or "saturdays"
+  * "sunday" or "sundays"
+  * "weekday" or "weekdays"
+  * "weekend" or ""weekends"
+* Relative day of week-based specifiers - includes or excludes based on the day of week relative to now
+  * "current day of the week"
+  * "current day of week"
+  * "this day of the week"
+  * "this day of week" 
+* Specified date - includes or excludes based on the specified date
+  * "date" - Takes up to 2 arguments
+    * The day in `yyyy/MM/dd` format if no second argument is provided
+    * Optionally the format to specify the first argument in
+    * Example: `date:2017/12/25` would include or exclude December 25, 2017
+    * Example: `date:20171225:yyyyMMdd` would include or exclude December 25, 2017
+* Holidays - includes or excludes based on if the window starts during a holiday
+  * "holiday" or "holidays" 
+    * Arguments form the jollyday hierarchy of holidays.  e.g. "us:nyc" would be holidays for New York City, USA
+    * If none is specified, it will choose based on locale.
+    * Countries supported are those supported in [jollyday](https://github.com/svendiedrichsen/jollyday/tree/master/src/main/resources/holidays)
+    * Example: `holiday:us:nyc` would be the holidays of New York City, USA
+    * Example: `holiday:hu` would be the holidays of Hungary
+     
+**Examples**
+
+Assume these are executed at noon.
+* A 1 hour window for the past 8 'current day of the week'
+  * <a href="#Temporal_Window_Width"><span style="color:red">1 hour window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 56 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">including this day of the week</span></a>
+* A 1 hour window for the past 8 tuesdays
+  * <a href="#Temporal_Window_Width"><span style="color:red">1 hour window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 56 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">including tuesdays</span></a>
+* A 30 minute window every tuesday at noon starting 14 days ago until now
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 14 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">including tuesdays</span></a>
+* A 30 minute window every day except holidays and weekends at noon starting 14 days ago until now
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 14 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">excluding holidays:us, weekends</span></a>
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 14 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">including weekdays excluding holidays:us, weekends</span></a>
+* A 30 minute window at noon every day from 7 days ago including saturdays and excluding weekends. 
+Because exclusions trump inclusions, the following will never yield any windows
+  * <a href="#Temporal_Window_Width"><span style="color:red">30 minute window</span></a> <a href="#Skip_distance"><span style="color:green">every 24 hours</span></a> <a href="#Total_Temporal_Duration"><span style="color:blue">from 7 days ago</span></a> <a href="#InclusionExclusion_specifiers"><span style="color:purple">including saturdays excluding weekends</span></a>
+
 ### Errors
-The most common result of incorrect PROFILE_GET arguments or Client configuration parameters is an empty result set, rather than an error.  The Client cannot effectively validate the arguments, because the Profiler configuration parameters may be changed and the profile itself does not store them.  The person doing the querying must carry forward the knowledge of the Profiler configuration parameters from the time of profile creation, and use corresponding PROFILE_GET arguments and Client configuration parameters when querying the data.
+The most common result of incorrect `PROFILE_GET` arguments or Client configuration parameters is an empty result set, rather than an error.
+The Client cannot effectively validate the arguments, because the Profiler configuration parameters may be changed and the profile itself does not store them.
+The person doing the querying must carry forward the knowledge of the Profiler configuration parameters from the time of profile creation, and use corresponding `PROFILE_GET` arguments and Client configuration parameters when querying the data.
 
 ### Examples
+
+The following are usage examples that show how the Stellar API can be used to read profiles generated by the [Metron Profiler](../metron-profiler).  This API would be used in conjunction with other Stellar functions like [`MAAS_MODEL_APPLY`](../../metron-platform/metron-common#maas_model_apply) to perform model scoring on streaming data.
+
+These examples assume a profile has been defined called 'snort-alerts' that tracks the number of Snort alerts associated with an IP address over time.  The profile definition might look similar to the following.
+
+```
+{
+  "profiles": [
+    {
+      "profile": "snort-alerts",
+      "foreach": "ip_src_addr",
+      "onlyif":  "source.type == 'snort'",
+      "update":  { "s": "STATS_ADD(s, 1)" },
+      "result":  "STATS_MEAN(s)"
+    }
+  ]
+}
+```
+
+During model scoring the entity being scored, in this case a particular IP address, will be known.  The following examples shows how this profile data might be retrieved.
 Retrieve all values of 'snort-alerts' from '10.0.0.1' over the past 4 hours.
 ```
-PROFILE_GET('snort-alerts', '10.0.0.1', 4, 'HOURS')
+PROFILE_GET('snort-alerts', '10.0.0.1', PROFILE_FIXED(4, 'HOURS'))
 ```
 
 Retrieve all values of 'snort-alerts' from '10.0.0.1' over the past 2 days.
 ```
-PROFILE_GET('snort-alerts', '10.0.0.1', 2, 'DAYS')
+PROFILE_GET('snort-alerts', '10.0.0.1', PROFILE_FIXED(2, 'DAYS'))
 ```
 
 If the profile had been defined to group the data by weekday versus weekend, then the following example would apply:
 
-Retrieve all values of 'snort-alerts' from '10.0.0.1' that occurred on 'weekdays' over the past month.
+Retrieve all values of 'snort-alerts' from '10.0.0.1' that occurred on 'weekdays' over the past 30 days.
 ```
-PROFILE_GET('snort-alerts', '10.0.0.1', 1, 'MONTHS', ['weekdays'] )
+PROFILE_GET('snort-alerts', '10.0.0.1', PROFILE_FIXED(30, 'DAYS'), ['weekdays'] )
 ```
 
 The client may need to use a configuration different from the current Client configuration settings.  For example, perhaps you are on a cluster shared with other analysts, and need to access a profile that was constructed 2 months ago using different period duration, while they are accessing more recent profiles constructed with the currently configured period duration.  For this situation, you may use the `config_overrides` argument:
 
 Retrieve all values of 'snort-alerts' from '10.0.0.1' over the past 2 days, with no `groupBy`, and overriding the usual global client configuration parameters for window duration.
 ```
-PROFILE_GET('profile1', 'entity1', 2, 'DAYS', [], {'profiler.client.period.duration' : '2', 'profiler.client.period.duration.units' : 'MINUTES'})
+PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(2, 'DAYS', {'profiler.client.period.duration' : '2', 'profiler.client.period.duration.units' : 'MINUTES'}), [])
 ```
 
-Retrieve all values of 'snort-alerts' from '10.0.0.1' that occurred on 'weekdays' over the past month, overriding the usual global client configuration parameters for window duration.
+Retrieve all values of 'snort-alerts' from '10.0.0.1' that occurred on 'weekdays' over the past 30 days, overriding the usual global client configuration parameters for window duration.
 ```
-PROFILE_GET('profile1', 'entity1', 1, 'MONTHS', ['weekdays'], {'profiler.client.period.duration' : '2', 'profiler.client.period.duration.units' : 'MINUTES'})
+PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(30, 'DAYS', {'profiler.client.period.duration' : '2', 'profiler.client.period.duration.units' : 'MINUTES'}), ['weekdays'] )
 ```
 
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/pom.xml b/metron-analytics/metron-profiler-client/pom.xml
index 77eb078..9b08be9 100644
--- a/metron-analytics/metron-profiler-client/pom.xml
+++ b/metron-analytics/metron-profiler-client/pom.xml
@@ -59,6 +59,11 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
+            <groupId>de.jollyday</groupId>
+            <artifactId>jollyday</artifactId>
+            <version>0.5.2</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>metron-profiler-common</artifactId>
             <version>${project.parent.version}</version>
@@ -236,6 +241,12 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>org.antlr</groupId>
+            <artifactId>antlr4-runtime</artifactId>
+            <version>${global_antlr_version}</version>
+        </dependency>
+
     </dependencies>
     <reporting>
         <plugins>
@@ -266,6 +277,22 @@
     <build>
         <plugins>
             <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <version>${global_antlr_version}</version>
+                <configuration>
+                  <outputDirectory>${basedir}/src/main/java</outputDirectory>
+                </configuration>
+                <executions>
+                  <execution>
+                    <goals>
+                      <goal>antlr4</goal>
+                    </goals>
+                  </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-shade-plugin</artifactId>
                 <version>${global_shade_version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/antlr4/org/apache/metron/profiler/client/window/generated/Window.g4
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/antlr4/org/apache/metron/profiler/client/window/generated/Window.g4 b/metron-analytics/metron-profiler-client/src/main/antlr4/org/apache/metron/profiler/client/window/generated/Window.g4
new file mode 100644
index 0000000..0090971
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/antlr4/org/apache/metron/profiler/client/window/generated/Window.g4
@@ -0,0 +1,136 @@
+/*
+ * 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.
+ */
+
+grammar Window;
+
+@header {
+//CHECKSTYLE:OFF
+/*
+ * 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.
+ */
+}
+COMMA : ',';
+COLON : ':';
+
+WINDOW : 'window' | 'windows';
+
+INCLUDE : 'include' | 'INCLUDE' | 'includes' | 'INCLUDES' | 'including' | 'INCLUDING';
+EXCLUDE : 'exclude' | 'EXCLUDE' | 'excludes' | 'EXCLUDES' | 'excluding' | 'EXCLUDING';
+
+FROM : 'FROM' | 'from' | 'STARTING FROM' | 'starting from';
+EVERY : 'EVERY' | 'every' | 'FOR EVERY' | 'for every';
+TO : 'TO' | 'to' | 'until' | 'UNTIL';
+AGO : 'AGO' | 'ago';
+
+NUMBER : FIRST_DIGIT DIGIT*;
+IDENTIFIER : [:][a-zA-Z0-9][a-zA-Z0-9_\.\-/]*;
+
+DAY_SPECIFIER : MONDAY | TUESDAY | WEDNESDAY | THURSDAY
+                       | FRIDAY | SATURDAY | SUNDAY
+                       | CURRENT_DAY_OF_WEEK
+                       | WEEKEND | WEEKDAY | HOLIDAYS
+                       | DATE
+                       ;
+
+TIME_UNIT : SECOND_UNIT | MINUTE_UNIT | HOUR_UNIT | DAY_UNIT ;
+
+WS : [ \r\t\u000C\n]+ -> skip;
+
+fragment SECOND_UNIT : 'SECOND' | 'second' | 'seconds' | 'SECONDS' | 'second(s)' | 'SECOND(S)';
+fragment MINUTE_UNIT : 'MINUTE' | 'minute' | 'minutes' | 'MINUTES' | 'minute(s)' | 'MINUTE(S)';
+fragment HOUR_UNIT : 'HOUR' | 'hour' | 'hours' | 'HOURS' | 'hour(s)' | 'HOUR(S)';
+fragment DAY_UNIT : 'DAY' | 'day' | 'days' | 'DAYS' | 'day(s)' | 'DAY(S)';
+fragment MONDAY : 'MONDAY' | 'monday' | 'MONDAYS' | 'mondays';
+fragment TUESDAY : 'TUESDAY' | 'tuesday' | 'TUESDAYS' | 'tuesdays';
+fragment WEDNESDAY : 'WEDNESDAY' | 'wednesday' | 'WEDNESDAYS' | 'wednesdays';
+fragment THURSDAY : 'THURSDAY' | 'thursday' | 'THURSDAYS' | 'thursdays';
+fragment FRIDAY : 'FRIDAY' | 'friday' | 'FRIDAYS' | 'fridays';
+fragment SATURDAY: 'SATURDAY' | 'saturday' | 'SATURDAYS' | 'saturdays';
+fragment SUNDAY : 'SUNDAY' | 'sunday' | 'SUNDAYS' | 'sundays';
+fragment CURRENT_DAY_OF_WEEK: 'this day of week' | 'THIS DAY OF WEEK' | 'this day of the week' | 'THIS DAY OF THE WEEK'
+                            | 'current day of week' | 'CURRENT DAY OF WEEK'
+                            | 'current day of the week' | 'CURRENT DAY OF THE WEEK';
+fragment WEEKEND : 'weekend' | 'WEEKEND' | 'weekends' | 'WEEKENDS';
+fragment WEEKDAY: 'weekday' | 'WEEKDAY' | 'weekdays' | 'WEEKDAYS';
+fragment HOLIDAYS: 'holiday' | 'HOLIDAY' | 'holidays' | 'HOLIDAYS';
+fragment DATE: 'date' | 'DATE';
+
+fragment DIGIT : '0'..'9';
+fragment FIRST_DIGIT : '1'..'9';
+
+window : window_expression EOF;
+
+window_expression : window_width including_specifier? excluding_specifier? #NonRepeatingWindow
+                  | window_width skip_distance duration including_specifier? excluding_specifier? #RepeatingWindow
+                  | duration #DenseWindow
+                  ;
+
+excluding_specifier : EXCLUDE specifier_list
+                    ;
+including_specifier : INCLUDE specifier_list
+                    ;
+
+specifier : day_specifier
+          | day_specifier specifier_arg_list
+          ;
+
+specifier_arg_list : identifier
+                   | identifier specifier_arg_list
+                    ;
+
+day_specifier : DAY_SPECIFIER ;
+
+identifier : NUMBER | IDENTIFIER
+          ;
+
+specifier_list : specifier
+               | specifier_list COMMA specifier
+               ;
+
+duration : FROM time_interval AGO? TO time_interval AGO? #FromToDuration
+         | FROM time_interval AGO? #FromDuration
+         ;
+
+skip_distance : EVERY time_interval #SkipDistance
+              ;
+
+window_width : time_interval WINDOW? #WindowWidth
+          ;
+
+time_interval : time_amount time_unit #TimeInterval
+              ;
+
+time_amount : NUMBER #TimeAmount
+            ;
+
+time_unit : TIME_UNIT #TimeUnit
+            ;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/Window.tokens
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/Window.tokens b/metron-analytics/metron-profiler-client/src/main/java/Window.tokens
new file mode 100644
index 0000000..b3390d2
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/Window.tokens
@@ -0,0 +1,17 @@
+COMMA=1
+COLON=2
+WINDOW=3
+INCLUDE=4
+EXCLUDE=5
+NOW=6
+FROM=7
+EVERY=8
+TO=9
+AGO=10
+NUMBER=11
+IDENTIFIER=12
+DAY_SPECIFIER=13
+TIME_UNIT=14
+WS=15
+','=1
+':'=2

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/WindowLexer.tokens
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/WindowLexer.tokens b/metron-analytics/metron-profiler-client/src/main/java/WindowLexer.tokens
new file mode 100644
index 0000000..b3390d2
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/WindowLexer.tokens
@@ -0,0 +1,17 @@
+COMMA=1
+COLON=2
+WINDOW=3
+INCLUDE=4
+EXCLUDE=5
+NOW=6
+FROM=7
+EVERY=8
+TO=9
+AGO=10
+NUMBER=11
+IDENTIFIER=12
+DAY_SPECIFIER=13
+TIME_UNIT=14
+WS=15
+','=1
+':'=2

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/IntervalPredicate.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/IntervalPredicate.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/IntervalPredicate.java
new file mode 100644
index 0000000..f2f10d4
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/IntervalPredicate.java
@@ -0,0 +1,106 @@
+/*
+ *
+ *  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.metron.profiler.client.stellar;
+
+import org.apache.commons.lang3.Range;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+/**
+ * A predicate applied to a type T which can be converted into a long which indicates whether it exists
+ * within a set of inclusive ranges of longs.  Generally these ranges may be thought of as timestamps.
+ * In this interpretation, it will let you quickly indicate whether a given timestamp is within a set of timestamp
+ * ranges.
+ *
+ * @param <T>
+ */
+public class IntervalPredicate<T> implements Predicate<T> {
+  private final List<Range<Long>> intervals;
+  private final Function<T, Long> timestampTransformer;
+
+  /**
+   * In the situation where we want longs directly.
+   */
+  public static final class Identity extends IntervalPredicate<Long> {
+
+    public Identity(List<Range<Long>> intervals) {
+      super(x -> x, intervals, Long.class);
+    }
+  }
+
+  /**
+   * Construct an interval predicate given a set of intervals and a function to convert T's to timestamps.
+   * Please please please understand that intervals MUST be sorted.
+   *
+   * @param timestampTransformer The function to convert T's to timestamps.
+   * @param intervals A sorted list of timestamp intervals.
+   * @param clazz
+   */
+  public IntervalPredicate(Function<T, Long> timestampTransformer, List<Range<Long>> intervals, Class<T> clazz) {
+    this.intervals = intervals;
+    this.timestampTransformer = timestampTransformer;
+  }
+
+  private boolean containsInclusive(Range<Long> interval, long ts) {
+    return interval.contains(ts) || interval.getMaximum() == ts;
+  }
+
+
+  /**
+   * A helpful interval comparator that looks sorts the intervals according to left-side.
+   */
+  public static final Comparator<Range<Long>> INTERVAL_COMPARATOR = (o1, o2) -> {
+      if(o1.getMinimum() == o2.getMinimum() && o1.getMaximum() == o2.getMaximum()) {
+        return 0;
+      }
+      else {
+        int ret = Long.compare(o1.getMinimum(), o2.getMinimum());
+        if(ret == 0) {
+          return Long.compare(o1.getMaximum(), o2.getMaximum());
+        }
+        else {
+          return ret;
+        }
+      }
+  };
+
+  /**
+   * Determine if x is in the set of intervals in O(log*n) time.
+   * @param x
+   * @return true if in the set of intervals and false otherwise.
+   */
+  @Override
+  public boolean test(T x) {
+    long ts = timestampTransformer.apply(x);
+    int pos = Collections.binarySearch(intervals, Range.is(ts), INTERVAL_COMPARATOR);
+    if(pos < 0) {
+      pos = -pos - 1;
+    }
+
+    Optional<Range<Long>> right = pos >= 0 && pos < intervals.size()?Optional.of(intervals.get(pos)):Optional.empty();
+    Optional<Range<Long>> left = pos - 1 >= 0 && pos - 1 < intervals.size()?Optional.of(intervals.get(pos - 1)):Optional.empty();
+    return (right.isPresent()?containsInclusive(right.get(),ts):false) || (left.isPresent()?containsInclusive(left.get(),ts):false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/WindowLookback.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/WindowLookback.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/WindowLookback.java
new file mode 100644
index 0000000..c7390fa
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/WindowLookback.java
@@ -0,0 +1,107 @@
+/*
+ *
+ *  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.metron.profiler.client.stellar;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.Stellar;
+import org.apache.metron.common.dsl.StellarFunction;
+import org.apache.metron.common.utils.ConversionUtils;
+import org.apache.metron.profiler.ProfilePeriod;
+import org.apache.metron.profiler.client.window.Window;
+import org.apache.metron.profiler.client.window.WindowProcessor;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+@Stellar(
+      namespace="PROFILE",
+      name="WINDOW",
+      description="The profiler periods associated with a window selector statement from an optional reference timestamp.",
+      params={
+        "windowSelector - The statement specifying the window to select.",
+        "now - Optional - The timestamp to use for now.",
+        "config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter " +
+                "of the same name. Default is the empty Map, meaning no overrides."
+      },
+      returns="The selected profile measurement periods.  These are ProfilePeriod objects."
+)
+public class WindowLookback implements StellarFunction {
+
+  private Cache<String, Window> windowCache;
+
+  @Override
+  public Object apply(List<Object> args, Context context) throws ParseException {
+    Optional<Map> configOverridesMap = Optional.empty();
+    long now = System.currentTimeMillis();
+    String windowSelector = Util.getArg(0, String.class, args);
+    if(args.size() > 1) {
+      Optional<Object> arg2 = Optional.ofNullable(args.get(1));
+      Optional<Object> mapArg = args.size() > 2?Optional.ofNullable(args.get(2)):Optional.empty();
+      if(!mapArg.isPresent() && arg2.isPresent() && arg2.get() instanceof Map) {
+        mapArg = arg2;
+      }
+
+      if(arg2.isPresent() && arg2.get() instanceof Number) {
+        now = ConversionUtils.convert(arg2.get(), Long.class);
+      }
+
+      if(mapArg.isPresent()) {
+        Map rawMap = ConversionUtils.convert(mapArg.get(), Map.class);
+        configOverridesMap = rawMap == null || rawMap.isEmpty() ? Optional.empty() : Optional.of(rawMap);
+      }
+
+    }
+    Map<String, Object> effectiveConfigs = Util.getEffectiveConfig(context, configOverridesMap.orElse(null));
+    Long tickDuration = ProfilerConfig.PROFILER_PERIOD.get(effectiveConfigs, Long.class);
+    TimeUnit tickUnit = TimeUnit.valueOf(ProfilerConfig.PROFILER_PERIOD_UNITS.get(effectiveConfigs, String.class));
+    Window w = null;
+    try {
+      w = windowCache.get(windowSelector, () -> WindowProcessor.process(windowSelector));
+    } catch (ExecutionException e) {
+      throw new IllegalStateException("Unable to process " + windowSelector + ": " + e.getMessage(), e);
+    }
+    long end = w.getEndMillis(now);
+    long start = w.getStartMillis(now);
+    IntervalPredicate<ProfilePeriod> intervalSelector = new IntervalPredicate<>(period -> period.getStartTimeMillis()
+                                                                               , w.toIntervals(now)
+                                                                               , ProfilePeriod.class
+                                                                               );
+    return ProfilePeriod.visitPeriods(start, end, tickDuration, tickUnit, Optional.of(intervalSelector), period -> period);
+  }
+
+  @Override
+  public void initialize(Context context) {
+    windowCache = CacheBuilder.newBuilder()
+                              .maximumSize(200)
+                              .expireAfterAccess(10, TimeUnit.MINUTES)
+                              .build();
+  }
+
+  @Override
+  public boolean isInitialized() {
+    return windowCache != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/Window.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/Window.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/Window.java
new file mode 100644
index 0000000..8f697aa
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/Window.java
@@ -0,0 +1,159 @@
+/*
+ *
+ *  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.metron.profiler.client.window;
+
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang3.Range;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+/**
+ * A window is intended to compute the set of window intervals across time based on a reference time.
+ * The intervals are specified using a Window selector statement, which is a quasi-natural language grammar.
+ * Windows are intended to compute the set of intervals relative to a timestamp.
+ */
+public class Window {
+  private Function<Long, Long> startMillis ;
+  private Function<Long, Long> endMillis;
+  private List<Function<Long, Predicate<Long>>> includes = new ArrayList<>();
+  private List<Function<Long, Predicate<Long>>> excludes = new ArrayList<>();
+  private Optional<Long> binWidth = Optional.empty();
+  private Optional<Long> skipDistance = Optional.empty();
+
+  /**
+   * Return the start of the interval relative to the timestamp passed.
+   * @param now
+   * @return
+   */
+  public long getStartMillis(long now) {
+    return startMillis.apply(now);
+  }
+
+  void setStartMillis(Function<Long, Long> startMillis) {
+    this.startMillis = startMillis;
+  }
+
+  /**
+   * Return the end of the interval relative to the timestamp passed.
+   * @param now
+   * @return
+   */
+  public Long getEndMillis(long now) {
+    return endMillis.apply(now);
+  }
+
+  void setEndMillis(Function<Long, Long> endMillis) {
+    this.endMillis = endMillis;
+  }
+
+  /**
+   * Get the set of inclusion predicates.  If any of these are true as applied to the window interval start time,
+   * then a field is included unless it's explicitly excluded.
+   * @param now
+   * @return
+   */
+  public Iterable<Predicate<Long>> getIncludes(long now) {
+    return Iterables.transform(includes, f -> f.apply(now));
+  }
+
+  void setIncludes(List<Function<Long, Predicate<Long>>> includes) {
+    this.includes = includes;
+  }
+
+  /**
+   * Get the set of exclusion predicates.  If any of these exclusion predicates are true as applied to the window
+   * interval start time, then the interval is excluded.  NOTE: Exclusions trump inclusions.
+   * @param now
+   * @return
+   */
+  public Iterable<Predicate<Long>> getExcludes(long now){
+    return Iterables.transform(excludes, f -> f.apply(now));
+  }
+
+  void setExcludes(List<Function<Long, Predicate<Long>>> excludes) {
+    this.excludes = excludes;
+  }
+
+  /**
+   * The bin width.  This is fixed regardless of relative time.
+   * @return
+   */
+  public Optional<Long> getBinWidth() {
+    return binWidth;
+  }
+
+  void setBinWidth(long binWidth) {
+    this.binWidth = Optional.of(binWidth);
+  }
+
+  /**
+   * The skip distance.  How long between interval windows that one must go.
+   * @return
+   */
+  public Optional<Long> getSkipDistance() {
+    return skipDistance;
+  }
+
+  void setSkipDistance(long skipDistance) {
+    this.skipDistance = Optional.of(skipDistance);
+  }
+
+  /**
+   * Compute the set of sorted (oldest to newest) window intervals relative to the passed timestamp
+   * given inclusion and exclusion predicates.
+   *
+   * @param now
+   * @return
+   */
+  public List<Range<Long>> toIntervals(long now) {
+    List<Range<Long>> intervals = new ArrayList<>();
+    long startMillis = getStartMillis(now);
+    long endMillis = getEndMillis(now);
+    Iterable<Predicate<Long>> includes = getIncludes(now);
+    Iterable<Predicate<Long>> excludes = getExcludes(now);
+    //if we don't have a skip distance, then we just skip past everything to make the window dense
+    long skipDistance = getSkipDistance().orElse(Long.MAX_VALUE);
+    //if we don't have a window width, then we want the window to be completely dense.
+    Optional<Long> binWidthOpt = getBinWidth();
+    long binWidth = binWidthOpt.isPresent()?binWidthOpt.get():endMillis-startMillis;
+
+    for(long left = startMillis;left >= 0 && left + binWidth <= endMillis;left += skipDistance) {
+      Range<Long> interval = Range.between(left, left + binWidth);
+      boolean include = includes.iterator().hasNext()?false:true;
+      for(Predicate<Long> inclusionPredicate : includes) {
+        include |= inclusionPredicate.test(left);
+      }
+      if(include) {
+        for(Predicate<Long> exclusionPredicate : excludes) {
+          include &= !exclusionPredicate.test(left);
+        }
+      }
+      if(include) {
+        intervals.add(interval);
+      }
+    }
+    return intervals;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/WindowProcessor.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/WindowProcessor.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/WindowProcessor.java
new file mode 100644
index 0000000..16a7513
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/WindowProcessor.java
@@ -0,0 +1,422 @@
+/*
+ *
+ *  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.metron.profiler.client.window;
+
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.metron.common.dsl.ErrorListener;
+import org.apache.metron.common.dsl.GrammarUtils;
+import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.Token;
+import org.apache.metron.common.utils.ConversionUtils;
+import org.apache.metron.profiler.client.window.generated.WindowBaseListener;
+import org.apache.metron.profiler.client.window.generated.WindowLexer;
+import org.apache.metron.profiler.client.window.generated.WindowParser;
+import org.apache.metron.profiler.client.window.predicates.DayPredicates;
+
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+
+/**
+ * The WindowProcessor instance provides the parser callbacks for the Window selector language.  This constructs
+ * a Window object to be used to compute sparse window intervals across time.
+ */
+public class WindowProcessor extends WindowBaseListener {
+  private Throwable throwable;
+  private Deque<Token<?>> stack;
+  private static final Token<Object> LIST_MARKER = new Token<>(null, Object.class);
+  private static final Token<Object> SPECIFIER_MARKER = new Token<>(null, Object.class);
+  private Window window;
+
+  public WindowProcessor() {
+    this.stack = new ArrayDeque<>();
+    this.window = new Window();
+  }
+
+  /**
+   * Retrieve the window constructed from the window selector statement.
+   * @return
+   */
+  public Window getWindow() {
+    return window;
+  }
+
+  private void enterList() {
+    stack.push(LIST_MARKER);
+  }
+
+  private List<Function<Long, Predicate<Long>>> getPredicates() {
+    LinkedList<Function<Long, Predicate<Long>>> predicates = new LinkedList<>();
+    while (true) {
+      Token<?> token = stack.pop();
+      if (token == LIST_MARKER) {
+        break;
+      } else {
+        predicates.addFirst((Function<Long, Predicate<Long>>) token.getValue());
+      }
+    }
+    return predicates;
+  }
+
+  /**
+   * If we see an identifier, an argument for an inclusion/exclusion predicate, then we want to just push it onto the
+   * stack without its ':'.
+   * @param ctx
+   */
+  @Override
+  public void exitIdentifier(WindowParser.IdentifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    stack.push(new Token<>(ctx.getText().substring(1), String.class));
+  }
+
+  /**
+   * When we enter a specifier then we want to push onto the stack the specifier marker so we know when
+   * the specifier parameters end.
+   * @param ctx
+   */
+  @Override
+  public void enterSpecifier(WindowParser.SpecifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    stack.push(SPECIFIER_MARKER);
+  }
+
+  /**
+   * Read the specifier params off the stack in FIFO order until we get to the specifier marker.  Now we can
+   * construct the specifier, which is a Function which constructs a Selector Predicate based on the args
+   * passed to the selector e.g. holidays:us:nyc would have 2 args us and nyc.
+   *
+   * @param ctx
+   */
+  @Override
+  public void exitSpecifier(WindowParser.SpecifierContext ctx) {
+    LinkedList<String> args = new LinkedList<>();
+
+    while (true) {
+      Token<?> token = stack.pop();
+      if (token == SPECIFIER_MARKER) {
+        break;
+      } else {
+        args.addFirst((String) token.getValue());
+      }
+    }
+    String specifier = args.removeFirst();
+    List<String> arg = args.size() > 0?args:new ArrayList<>();
+    Function<Long, Predicate<Long>> predicate = null;
+    try {
+      if (specifier.equals("THIS DAY OF THE WEEK") || specifier.equals("THIS DAY OF WEEK")) {
+        predicate = now -> DayPredicates.dayOfWeekPredicate(DayPredicates.getDayOfWeek(now));
+      } else {
+        final Predicate<Long> dayOfWeekPredicate = DayPredicates.create(specifier, arg);
+        predicate = now -> dayOfWeekPredicate;
+      }
+      stack.push(new Token<>(predicate, Function.class));
+    }
+    catch(Throwable t) {
+      throwable = t;
+    }
+  }
+
+  /**
+   * Normalize the day specifier e.g. tuesdays -> tuesday and push onto the stack.
+   * @param ctx
+   */
+  @Override
+  public void exitDay_specifier(WindowParser.Day_specifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    String specifier = ctx.getText().toUpperCase();
+    if(specifier.length() == 0 && ctx.exception != null){
+      IllegalStateException ise = new IllegalStateException("Invalid day specifier: " + ctx.getStart().getText(), ctx.exception);
+      throwable = ise;
+      throw ise;
+    }
+    if(specifier.endsWith("S")) {
+      specifier = specifier.substring(0, specifier.length() - 1);
+    }
+    stack.push(new Token<>(specifier, String.class));
+  }
+
+  /**
+   * When we're beginning an exclusion specifier list, then we push the list token so we
+   * know when we're done processing
+   * @param ctx
+   */
+  @Override
+  public void enterExcluding_specifier(WindowParser.Excluding_specifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    enterList();
+  }
+
+  /**
+   * And when we're done with the exclusions specifier, then we set the exclusions
+   * to the predicates we've put on the stack.
+   * @param ctx
+   */
+  @Override
+  public void exitExcluding_specifier(WindowParser.Excluding_specifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    window.setExcludes(getPredicates());
+  }
+
+  /**
+   * When we're beginning an inclusion specifier list, then we push the list token so we
+   * know when we're done processing
+   * @param ctx
+   */
+  @Override
+  public void enterIncluding_specifier(WindowParser.Including_specifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    enterList();
+  }
+
+  /**
+   * And when we're done with the inclusions specifier, then we set the exclusions
+   * to the predicates we've put on the stack.
+   * @param ctx
+   */
+  @Override
+  public void exitIncluding_specifier(WindowParser.Including_specifierContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    window.setIncludes(getPredicates());
+  }
+
+  private void setFromTo(long from, long to) {
+    window.setEndMillis(now -> now - Math.min(to, from));
+    window.setStartMillis(now -> now - Math.max(from, to));
+  }
+
+  /**
+   * If we have a total time interval that we've specified, then we want to set the interval.
+   * NOTE: the interval will be set based on the smallest to largest being the start and end time respectively.
+   * Thus 'from 1 hour ago to 1 day ago' and 'from 1 day ago to 1 hour ago' are equivalent.
+   * @param ctx
+   */
+  @Override
+  public void exitFromToDuration(org.apache.metron.profiler.client.window.generated.WindowParser.FromToDurationContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    Token<?> toInterval = stack.pop();
+    Token<?> fromInterval = stack.pop();
+    Long to = (Long)toInterval.getValue();
+    Long from = (Long)fromInterval.getValue();
+    setFromTo(from, to);
+  }
+
+  /**
+   * When we've done specifying a from, then we want to set it.
+   * @param ctx
+   */
+  @Override
+  public void exitFromDuration(org.apache.metron.profiler.client.window.generated.WindowParser.FromDurationContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    Token<?> timeInterval = stack.pop();
+    Long from = (Long)timeInterval.getValue();
+    setFromTo(from, 0);
+  }
+
+  /**
+   * We've set a skip distance.
+   * @param ctx
+   */
+  @Override
+  public void exitSkipDistance(org.apache.metron.profiler.client.window.generated.WindowParser.SkipDistanceContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    Token<?> timeInterval = stack.pop();
+    Long width = (Long)timeInterval.getValue();
+    window.setSkipDistance(width);
+  }
+
+  /**
+   * We've set a window width.
+   * @param ctx
+   */
+  @Override
+  public void exitWindowWidth(org.apache.metron.profiler.client.window.generated.WindowParser.WindowWidthContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    Token<?> timeInterval = stack.pop();
+    Long width = (Long)timeInterval.getValue();
+    window.setBinWidth(width);
+    window.setStartMillis(now -> now - width);
+    window.setEndMillis(now -> now);
+  }
+
+  /**
+   * We've set a time interval, which is a value along with a unit.
+   * @param ctx
+   */
+  @Override
+  public void exitTimeInterval(org.apache.metron.profiler.client.window.generated.WindowParser.TimeIntervalContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    Token<?> timeUnit = stack.pop();
+    Token<?> timeDuration = stack.pop();
+    long duration = ConversionUtils.convert(timeDuration.getValue(), Long.class);
+    TimeUnit unit = (TimeUnit) timeUnit.getValue();
+    stack.push(new Token<>(unit.toMillis(duration), Long.class));
+  }
+
+  /**
+   * We've set a time amount, which is integral.
+   * @param ctx
+   */
+  @Override
+  public void exitTimeAmount(org.apache.metron.profiler.client.window.generated.WindowParser.TimeAmountContext ctx) {
+    if(checkForException(ctx)) {
+      return;
+    }
+    if(ctx.getText().length() == 0) {
+      throwable = new IllegalStateException("Unable to process empty string.");
+      return;
+    }
+    long duration = Long.parseLong(ctx.getText());
+    stack.push(new Token<>(duration, Long.class));
+  }
+
+  /**
+   * We've set a time unit.  We support the timeunits provided by java.util.concurrent.TimeUnit
+   * @param ctx
+   */
+  @Override
+  public void exitTimeUnit(org.apache.metron.profiler.client.window.generated.WindowParser.TimeUnitContext ctx) {
+    checkForException(ctx);
+    switch(normalizeTimeUnit(ctx.getText())) {
+      case "DAY":
+        stack.push(new Token<>(TimeUnit.DAYS, TimeUnit.class));
+        break;
+      case "HOUR":
+        stack.push(new Token<>(TimeUnit.HOURS, TimeUnit.class));
+        break;
+      case "MINUTE":
+        stack.push(new Token<>(TimeUnit.MINUTES, TimeUnit.class));
+        break;
+      case "SECOND":
+        stack.push(new Token<>(TimeUnit.SECONDS, TimeUnit.class));
+        break;
+      default:
+        throw new IllegalStateException("Unsupported time unit: " + ctx.getText()
+                + ".  Supported units are limited to: day, hour, minute, second "
+                + "with any pluralization or capitalization.");
+    }
+  }
+
+  private boolean checkForException(ParserRuleContext ctx) {
+    if(throwable != null)  {
+      return true;
+    }
+    else if(ctx.exception != null) {
+      return true;
+    }
+    return false;
+  }
+
+  private static String normalizeTimeUnit(String s) {
+    String ret = s.toUpperCase().replaceAll("[^A-Z]", "");
+    if(ret.endsWith("S")) {
+      return ret.substring(0, ret.length() - 1);
+    }
+    return ret;
+  }
+
+  private static TokenStream createTokenStream(String statement) {
+    if (statement == null || isEmpty(statement.trim())) {
+      return null;
+    }
+    statement = statement.trim();
+    ANTLRInputStream input = new ANTLRInputStream(statement);
+    WindowLexer lexer = new WindowLexer(input);
+    lexer.removeErrorListeners();
+    lexer.addErrorListener(new ErrorListener());
+    TokenStream tokens = new CommonTokenStream(lexer);
+    return tokens;
+  }
+
+  private static WindowParser createParser(TokenStream tokens, Optional<WindowProcessor> windowProcessor) {
+    WindowParser parser = new WindowParser(tokens);
+    if(windowProcessor.isPresent()) {
+      parser.addParseListener(windowProcessor.get());
+    }
+    parser.removeErrorListeners();
+    parser.addErrorListener(new ErrorListener());
+    return parser;
+  }
+
+  /**
+   * Create a reusable Window object (parameterized by time) from a statement specifying the window intervals
+   * conforming to the Window grammar.
+   *
+   * @param statement
+   * @return
+   * @throws ParseException
+   */
+  public static Window process(String statement) throws ParseException {
+    TokenStream tokens = createTokenStream(statement);
+    if(tokens == null) {
+      return null;
+    }
+    WindowProcessor treeBuilder = new WindowProcessor();
+    WindowParser parser = createParser(tokens, Optional.of(treeBuilder));
+    parser.window();
+    if(treeBuilder.throwable != null) {
+      throw new ParseException(treeBuilder.throwable.getMessage(), treeBuilder.throwable);
+    }
+    return treeBuilder.getWindow();
+  }
+
+  /**
+   * Create a textual representation of the syntax tree.  This is useful for those intrepid souls
+   * who wish to extend the window selector language.  God speed.
+   * @param statement
+   * @return  A string representation of the syntax tree.
+   */
+  public static String syntaxTree(String statement) {
+    TokenStream tokens = createTokenStream(statement);
+    if(tokens == null) {
+      return null;
+    }
+    WindowParser parser = createParser(tokens, Optional.empty());
+    ParseTree tree = parser.window();
+    return GrammarUtils.toSyntaxTree(tree) ;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/84d34719/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowBaseListener.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowBaseListener.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowBaseListener.java
new file mode 100644
index 0000000..163cf1f
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/window/generated/WindowBaseListener.java
@@ -0,0 +1,276 @@
+// Generated from org/apache/metron/profiler/client/window/generated/Window.g4 by ANTLR 4.5
+package org.apache.metron.profiler.client.window.generated;
+
+//CHECKSTYLE:OFF
+/*
+ * 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.
+ */
+
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.ErrorNode;
+import org.antlr.v4.runtime.tree.TerminalNode;
+
+/**
+ * This class provides an empty implementation of {@link WindowListener},
+ * which can be extended to create a listener which only needs to handle a subset
+ * of the available methods.
+ */
+public class WindowBaseListener implements WindowListener {
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterWindow(WindowParser.WindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitWindow(WindowParser.WindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterNonRepeatingWindow(WindowParser.NonRepeatingWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitNonRepeatingWindow(WindowParser.NonRepeatingWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterRepeatingWindow(WindowParser.RepeatingWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitRepeatingWindow(WindowParser.RepeatingWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterDenseWindow(WindowParser.DenseWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitDenseWindow(WindowParser.DenseWindowContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterExcluding_specifier(WindowParser.Excluding_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitExcluding_specifier(WindowParser.Excluding_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterIncluding_specifier(WindowParser.Including_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitIncluding_specifier(WindowParser.Including_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterSpecifier(WindowParser.SpecifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitSpecifier(WindowParser.SpecifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterSpecifier_arg_list(WindowParser.Specifier_arg_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitSpecifier_arg_list(WindowParser.Specifier_arg_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterDay_specifier(WindowParser.Day_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitDay_specifier(WindowParser.Day_specifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterIdentifier(WindowParser.IdentifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitIdentifier(WindowParser.IdentifierContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterSpecifier_list(WindowParser.Specifier_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitSpecifier_list(WindowParser.Specifier_listContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterFromToDuration(WindowParser.FromToDurationContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitFromToDuration(WindowParser.FromToDurationContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterFromDuration(WindowParser.FromDurationContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitFromDuration(WindowParser.FromDurationContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterSkipDistance(WindowParser.SkipDistanceContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitSkipDistance(WindowParser.SkipDistanceContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterWindowWidth(WindowParser.WindowWidthContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitWindowWidth(WindowParser.WindowWidthContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterTimeInterval(WindowParser.TimeIntervalContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitTimeInterval(WindowParser.TimeIntervalContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterTimeAmount(WindowParser.TimeAmountContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitTimeAmount(WindowParser.TimeAmountContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterTimeUnit(WindowParser.TimeUnitContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitTimeUnit(WindowParser.TimeUnitContext ctx) { }
+
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void enterEveryRule(ParserRuleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void exitEveryRule(ParserRuleContext ctx) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void visitTerminal(TerminalNode node) { }
+	/**
+	 * {@inheritDoc}
+	 *
+	 * <p>The default implementation does nothing.</p>
+	 */
+	@Override public void visitErrorNode(ErrorNode node) { }
+}
\ No newline at end of file