From 1fcfd6d26bb59f2779f39dc9345d29f0eea63c56 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 21 Apr 2020 10:35:54 -0700 Subject: [PATCH 1/4] Lazy initialization of SettableByteEntityReader to avoid NPE --- .../SeekableStreamIndexTaskIOConfig.java | 1 + .../SeekableStreamIndexTaskRunner.java | 11 +- .../seekablestream/StreamChunkParser.java | 33 +++- .../seekablestream/StreamChunkParserTest.java | 183 ++++++++++++++++++ 4 files changed, 218 insertions(+), 10 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index 9ac35a9fab9e..ae2cd88bf8f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -126,6 +126,7 @@ private InputFormat getGivenInputFormat() return inputFormat; } + @Nullable public InputFormat getInputFormat(ParseSpec parseSpec) { return inputFormat == null ? Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat() : inputFormat; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 1577833d5fdd..bcdcecce304a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -202,6 +202,7 @@ public enum Status private final SeekableStreamIndexTaskIOConfig ioConfig; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; + @Nullable private final InputFormat inputFormat; @Nullable private final InputRowParser parser; @@ -372,12 +373,10 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception // Now we can initialize StreamChunkReader with the given toolbox. final StreamChunkParser parser = new StreamChunkParser( this.parser, - new SettableByteEntityReader( - inputFormat, - inputRowSchema, - task.getDataSchema().getTransformSpec(), - toolbox.getIndexingTmpDir() - ) + inputFormat, + inputRowSchema, + task.getDataSchema().getTransformSpec(), + toolbox.getIndexingTmpDir() ); initializeSequences(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 3f9b28138d82..8c4678867904 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -19,12 +19,19 @@ package org.apache.druid.indexing.seekablestream; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -38,12 +45,30 @@ class StreamChunkParser { @Nullable private final InputRowParser parser; - private final SettableByteEntityReader byteEntityReader; + private final Supplier lazyByteEntityReaderSupplier; // lazy initializer - StreamChunkParser(@Nullable InputRowParser parser, SettableByteEntityReader byteEntityReader) + /** + * Either parser or inputFormat shouldn't be null. + */ + StreamChunkParser( + @Nullable InputRowParser parser, + @Nullable InputFormat inputFormat, + InputRowSchema inputRowSchema, + TransformSpec transformSpec, + File indexingTmpDir + ) { + if (parser == null && inputFormat == null) { + throw new IAE("Either parser or inputFormat shouldn't be set"); + } this.parser = parser; - this.byteEntityReader = byteEntityReader; + // Create a lazy initializer since it will fail to create a SettableByteEntityReader if inputFormat is null + this.lazyByteEntityReaderSupplier = Suppliers.memoize(() -> new SettableByteEntityReader( + inputFormat, + inputRowSchema, + transformSpec, + indexingTmpDir + )); } List parse(List streamChunk) throws IOException @@ -51,7 +76,7 @@ List parse(List streamChunk) throws IOException if (parser != null) { return parseWithParser(parser, streamChunk); } else { - return parseWithInputFormat(byteEntityReader, streamChunk); + return parseWithInputFormat(lazyByteEntityReaderSupplier.get(), streamChunk); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java new file mode 100644 index 000000000000..7fdfe79e5d9c --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java @@ -0,0 +1,183 @@ +/* + * 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.druid.indexing.seekablestream; + +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.segment.transform.TransformSpec; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; + +public class StreamChunkParserTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec(null, null, null); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testWithParserAndNullInputformatParseProperly() throws IOException + { + final InputRowParser parser = new StringInputRowParser( + new NotConvertibleToInputFormatParseSpec(), + StringUtils.UTF8_STRING + ); + final StreamChunkParser chunkParser = new StreamChunkParser( + parser, + // Set nulls for all parameters below since inputFormat will be never used. + null, + null, + null, + null + ); + final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; + List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); + Assert.assertEquals(1, parsedRows.size()); + InputRow row = parsedRows.get(0); + Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); + Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); + Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); + } + + @Test + public void testWithNullParserAndInputformatParseProperly() throws IOException + { + final JsonInputFormat inputFormat = new JsonInputFormat(JSONPathSpec.DEFAULT, Collections.emptyMap()); + final StreamChunkParser chunkParser = new StreamChunkParser( + null, + inputFormat, + new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, Collections.emptyList()), + TransformSpec.NONE, + temporaryFolder.newFolder() + ); + final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; + List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); + Assert.assertEquals(1, parsedRows.size()); + InputRow row = parsedRows.get(0); + Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); + Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); + Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); + } + + @Test + public void testWithNullParserAndNullInputformatFailToCreateParser() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Either parser or inputFormat shouldn't be set"); + final StreamChunkParser chunkParser = new StreamChunkParser( + null, + null, + null, + null, + null + ); + } + + @Test + public void testBothParserAndInputFormatParseProperlyUsingParser() throws IOException + { + final TrackingStringInputRowParser parser = new TrackingStringInputRowParser( + new NotConvertibleToInputFormatParseSpec(), + StringUtils.UTF8_STRING + ); + final JsonInputFormat inputFormat = new JsonInputFormat(JSONPathSpec.DEFAULT, Collections.emptyMap()); + final StreamChunkParser chunkParser = new StreamChunkParser( + parser, + inputFormat, + new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, Collections.emptyList()), + TransformSpec.NONE, + temporaryFolder.newFolder() + ); + final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; + List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); + Assert.assertEquals(1, parsedRows.size()); + InputRow row = parsedRows.get(0); + Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); + Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); + Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); + Assert.assertTrue(parser.used); + } + + private static class NotConvertibleToInputFormatParseSpec extends JSONParseSpec + { + private NotConvertibleToInputFormatParseSpec() + { + super( + TIMESTAMP_SPEC, + DimensionsSpec.EMPTY, + JSONPathSpec.DEFAULT, + Collections.emptyMap() + ); + } + + @Override + public InputFormat toInputFormat() + { + return null; + } + } + + private static class TrackingStringInputRowParser extends StringInputRowParser + { + private boolean used; + + private TrackingStringInputRowParser(ParseSpec parseSpec, String encoding) + { + super(parseSpec, encoding); + } + + @Override + public List parseBatch(ByteBuffer input) + { + used = true; + return super.parseBatch(input); + } + + @Override + public InputRow parse(String input) + { + used = true; + return super.parse(input); + } + } +} From d452d1c018d435968b189c6274b79442a08f863d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 21 Apr 2020 10:55:11 -0700 Subject: [PATCH 2/4] toInputFormat for tsv --- .../apache/druid/data/input/impl/DelimitedParseSpec.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java index 5940e70e11fd..3ee0f71f62bd 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.java.util.common.parsers.DelimitedParser; import org.apache.druid.java.util.common.parsers.Parser; @@ -123,6 +124,12 @@ public Parser makeParser() ); } + @Override + public InputFormat toInputFormat() + { + return new DelimitedInputFormat(columns, listDelimiter, delimiter, hasHeaderRow, null, skipHeaderRows); + } + @Override public ParseSpec withTimestampSpec(TimestampSpec spec) { From a1acd298348fdf2eacf91b40dad5936dd96d57b4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 21 Apr 2020 13:59:51 -0700 Subject: [PATCH 3/4] address comments --- .../seekablestream/StreamChunkParser.java | 30 ++++++++-------- .../seekablestream/StreamChunkParserTest.java | 36 +++++++++---------- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 8c4678867904..6061aff7423b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -19,8 +19,6 @@ package org.apache.druid.indexing.seekablestream; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; @@ -45,7 +43,8 @@ class StreamChunkParser { @Nullable private final InputRowParser parser; - private final Supplier lazyByteEntityReaderSupplier; // lazy initializer + @Nullable + private final SettableByteEntityReader byteEntityReader; /** * Either parser or inputFormat shouldn't be null. @@ -59,24 +58,27 @@ class StreamChunkParser ) { if (parser == null && inputFormat == null) { - throw new IAE("Either parser or inputFormat shouldn't be set"); + throw new IAE("Either parser or inputFormat should be set"); } this.parser = parser; - // Create a lazy initializer since it will fail to create a SettableByteEntityReader if inputFormat is null - this.lazyByteEntityReaderSupplier = Suppliers.memoize(() -> new SettableByteEntityReader( - inputFormat, - inputRowSchema, - transformSpec, - indexingTmpDir - )); + if (inputFormat != null) { + this.byteEntityReader = new SettableByteEntityReader( + inputFormat, + inputRowSchema, + transformSpec, + indexingTmpDir + ); + } else { + this.byteEntityReader = null; + } } List parse(List streamChunk) throws IOException { - if (parser != null) { - return parseWithParser(parser, streamChunk); + if (byteEntityReader != null) { + return parseWithInputFormat(byteEntityReader, streamChunk); } else { - return parseWithInputFormat(lazyByteEntityReaderSupplier.get(), streamChunk); + return parseWithParser(parser, streamChunk); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java index 7fdfe79e5d9c..b24e5394a14c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java @@ -20,6 +20,8 @@ package org.apache.druid.indexing.seekablestream; import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; @@ -27,7 +29,6 @@ import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; -import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; @@ -40,10 +41,13 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; +import java.util.Map; public class StreamChunkParserTest { @@ -103,7 +107,7 @@ public void testWithNullParserAndInputformatParseProperly() throws IOException public void testWithNullParserAndNullInputformatFailToCreateParser() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Either parser or inputFormat shouldn't be set"); + expectedException.expectMessage("Either parser or inputFormat should be set"); final StreamChunkParser chunkParser = new StreamChunkParser( null, null, @@ -114,13 +118,16 @@ public void testWithNullParserAndNullInputformatFailToCreateParser() } @Test - public void testBothParserAndInputFormatParseProperlyUsingParser() throws IOException + public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws IOException { - final TrackingStringInputRowParser parser = new TrackingStringInputRowParser( + final InputRowParser parser = new StringInputRowParser( new NotConvertibleToInputFormatParseSpec(), StringUtils.UTF8_STRING ); - final JsonInputFormat inputFormat = new JsonInputFormat(JSONPathSpec.DEFAULT, Collections.emptyMap()); + final TrackingJsonInputFormat inputFormat = new TrackingJsonInputFormat( + JSONPathSpec.DEFAULT, + Collections.emptyMap() + ); final StreamChunkParser chunkParser = new StreamChunkParser( parser, inputFormat, @@ -135,7 +142,7 @@ public void testBothParserAndInputFormatParseProperlyUsingParser() throws IOExce Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); - Assert.assertTrue(parser.used); + Assert.assertTrue(inputFormat.used); } private static class NotConvertibleToInputFormatParseSpec extends JSONParseSpec @@ -157,27 +164,20 @@ public InputFormat toInputFormat() } } - private static class TrackingStringInputRowParser extends StringInputRowParser + private static class TrackingJsonInputFormat extends JsonInputFormat { private boolean used; - private TrackingStringInputRowParser(ParseSpec parseSpec, String encoding) - { - super(parseSpec, encoding); - } - - @Override - public List parseBatch(ByteBuffer input) + private TrackingJsonInputFormat(@Nullable JSONPathSpec flattenSpec, @Nullable Map featureSpec) { - used = true; - return super.parseBatch(input); + super(flattenSpec, featureSpec); } @Override - public InputRow parse(String input) + public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory) { used = true; - return super.parse(input); + return super.createReader(inputRowSchema, source, temporaryDirectory); } } } From 1d3f79602e0ecc40bdc695592e20167cc10682ee Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 23 Apr 2020 17:08:02 -0700 Subject: [PATCH 4/4] common code --- .../seekablestream/StreamChunkParserTest.java | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java index b24e5394a14c..76a02db31385 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java @@ -74,13 +74,7 @@ public void testWithParserAndNullInputformatParseProperly() throws IOException null, null ); - final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; - List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); - Assert.assertEquals(1, parsedRows.size()); - InputRow row = parsedRows.get(0); - Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); - Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); - Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); + parseAndAssertResult(chunkParser); } @Test @@ -94,13 +88,7 @@ public void testWithNullParserAndInputformatParseProperly() throws IOException TransformSpec.NONE, temporaryFolder.newFolder() ); - final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; - List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); - Assert.assertEquals(1, parsedRows.size()); - InputRow row = parsedRows.get(0); - Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); - Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); - Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); + parseAndAssertResult(chunkParser); } @Test @@ -135,6 +123,12 @@ public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws I TransformSpec.NONE, temporaryFolder.newFolder() ); + parseAndAssertResult(chunkParser); + Assert.assertTrue(inputFormat.used); + } + + private void parseAndAssertResult(StreamChunkParser chunkParser) throws IOException + { final String json = "{\"timestamp\": \"2020-01-01\", \"dim\": \"val\", \"met\": \"val2\"}"; List parsedRows = chunkParser.parse(Collections.singletonList(json.getBytes(StringUtils.UTF8_STRING))); Assert.assertEquals(1, parsedRows.size()); @@ -142,7 +136,6 @@ public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws I Assert.assertEquals(DateTimes.of("2020-01-01"), row.getTimestamp()); Assert.assertEquals("val", Iterables.getOnlyElement(row.getDimension("dim"))); Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met"))); - Assert.assertTrue(inputFormat.used); } private static class NotConvertibleToInputFormatParseSpec extends JSONParseSpec